You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sa...@apache.org on 2019/01/29 14:46:41 UTC

[01/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Repository: hive
Updated Branches:
  refs/heads/master 974708336 -> 71dfd1d11


http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index c0b1d87..b43fb5e 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -3521,6 +3521,29 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                                                     new AddPartitionEvent(tbl, newParts, true, this));
         }
 
+        if (!listeners.isEmpty()) {
+          MetaStoreListenerNotifier.notifyEvent(listeners,
+                  EventType.ADD_PARTITION,
+                  new AddPartitionEvent(tbl, newParts, true, this),
+                  null,
+                  transactionalListenerResponses, ms);
+
+          if (!existingParts.isEmpty()) {
+            // The request has succeeded but we failed to add these partitions.
+            MetaStoreListenerNotifier.notifyEvent(listeners,
+                    EventType.ADD_PARTITION,
+                    new AddPartitionEvent(tbl, existingParts, false, this),
+                    null, null, ms);
+          }
+        }
+
+        // Update partition column statistics if available
+        for (Partition newPart : newParts) {
+          if (newPart.isSetColStats()) {
+            updatePartitonColStatsInternal(tbl, newPart.getColStats(), null, newPart.getWriteId());
+          }
+        }
+
         success = ms.commitTransaction();
       } finally {
         if (!success) {
@@ -3533,24 +3556,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
                                                   new AddPartitionEvent(tbl, parts, false, this),
                                                   null, null, ms);
           }
-        } else {
-          if (!listeners.isEmpty()) {
-            MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                  EventType.ADD_PARTITION,
-                                                  new AddPartitionEvent(tbl, newParts, true, this),
-                                                  null,
-                                                  transactionalListenerResponses, ms);
-
-            if (!existingParts.isEmpty()) {
-              // The request has succeeded but we failed to add these partitions.
-              MetaStoreListenerNotifier.notifyEvent(listeners,
-                                                    EventType.ADD_PARTITION,
-                                                    new AddPartitionEvent(tbl, existingParts, false, this),
-                                                    null, null, ms);
-            }
-          }
         }
       }
+
       return newParts;
     }
 
@@ -6047,12 +6055,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
               EventType.UPDATE_PARTITION_COLUMN_STAT,
-              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, validWriteIds, writeId, this));
+              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, tbl, validWriteIds,
+                      writeId, this));
           }
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners,
               EventType.UPDATE_PARTITION_COLUMN_STAT,
-              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, validWriteIds, writeId, this));
+              new UpdatePartitionColumnStatEvent(colStats, partVals, parameters, tbl, validWriteIds,
+                      writeId, this));
           }
         }
         committed = getMS().commitTransaction();
@@ -6336,25 +6346,66 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public List<Partition> get_partitions_by_names(final String dbName, final String tblName,
                                                    final List<String> partNames) throws TException {
+      return get_partitions_by_names(dbName, tblName, partNames, false);
+    }
 
-      String[] parsedDbName = parseDbName(dbName, conf);
+    @Override
+    public GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest gpbnr)
+            throws TException {
+      List<Partition> partitions = get_partitions_by_names(gpbnr.getDb_name(),
+              gpbnr.getTbl_name(), gpbnr.getNames(),
+              gpbnr.isSetGet_col_stats() && gpbnr.isGet_col_stats());
+      return new GetPartitionsByNamesResult(partitions);
+    }
+
+    public List<Partition> get_partitions_by_names(final String dbName, final String tblName,
+           final List<String> partNames, boolean getColStats) throws TException {
+
+      String[] dbNameParts = parseDbName(dbName, conf);
+      String parsedCatName = dbNameParts[CAT_NAME];
+      String parsedDbName = dbNameParts[DB_NAME];
       List<Partition> ret = null;
       Exception ex = null;
-
-      startTableFunction("get_partitions_by_names", parsedDbName[CAT_NAME], parsedDbName[DB_NAME],
-          tblName);
+      boolean success = false;
+      startTableFunction("get_partitions_by_names", parsedCatName, parsedDbName,
+              tblName);
       try {
-        authorizeTableForPartitionMetadata(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
+        getMS().openTransaction();
+        authorizeTableForPartitionMetadata(parsedCatName, parsedDbName, tblName);
 
-        fireReadTablePreEvent(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName);
+        fireReadTablePreEvent(parsedCatName, parsedDbName, tblName);
 
-        ret = getMS().getPartitionsByNames(parsedDbName[CAT_NAME], parsedDbName[DB_NAME], tblName,
-            partNames);
+        ret = getMS().getPartitionsByNames(parsedCatName, parsedDbName, tblName, partNames);
         ret = FilterUtils.filterPartitionsIfEnabled(isServerFilterEnabled, filterHook, ret);
+
+        // If requested add column statistics in each of the partition objects
+        if (getColStats) {
+          Table table = getTable(parsedCatName, parsedDbName, tblName);
+          // Since each partition may have stats collected for different set of columns, we
+          // request them separately.
+          for (Partition part: ret) {
+            String partName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
+            List<ColumnStatistics> partColStatsList =
+                    getMS().getPartitionColumnStatistics(parsedCatName, parsedDbName, tblName,
+                            Collections.singletonList(partName),
+                            StatsSetupConst.getColumnsHavingStats(part.getParameters()));
+            if (partColStatsList != null && !partColStatsList.isEmpty()) {
+              ColumnStatistics partColStats = partColStatsList.get(0);
+              if (partColStats != null) {
+                part.setColStats(partColStats);
+              }
+            }
+          }
+        }
+
+        success = getMS().commitTransaction();
       } catch (Exception e) {
         ex = e;
         rethrowException(e);
       } finally {
+        if (!success) {
+          getMS().rollbackTransaction();
+        }
         endFunction("get_partitions_by_names", ret != null, ex, tblName);
       }
       return ret;

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/UpdatePartitionColumnStatEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/UpdatePartitionColumnStatEvent.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/UpdatePartitionColumnStatEvent.java
index a61b98c..094f799 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/UpdatePartitionColumnStatEvent.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/events/UpdatePartitionColumnStatEvent.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.Table;
 
 import java.util.List;
 import java.util.Map;
@@ -38,6 +39,7 @@ public class UpdatePartitionColumnStatEvent extends ListenerEvent {
   private long writeId;
   private Map<String, String> parameters;
   private List<String> partVals;
+  private Table tableObj;
 
   /**
    * @param statsObj Columns statistics Info.
@@ -48,13 +50,15 @@ public class UpdatePartitionColumnStatEvent extends ListenerEvent {
    * @param handler handler that is firing the event
    */
   public UpdatePartitionColumnStatEvent(ColumnStatistics statsObj, List<String> partVals, Map<String, String> parameters,
-                                    String validWriteIds, long writeId, IHMSHandler handler) {
+                                        Table tableObj, String validWriteIds, long writeId,
+                                        IHMSHandler handler) {
     super(true, handler);
     this.partColStats = statsObj;
     this.validWriteIds = validWriteIds;
     this.writeId = writeId;
     this.parameters = parameters;
     this.partVals = partVals;
+    this.tableObj = tableObj;
   }
 
   /**
@@ -62,13 +66,15 @@ public class UpdatePartitionColumnStatEvent extends ListenerEvent {
    * @param partVals partition names
    * @param handler handler that is firing the event
    */
-  public UpdatePartitionColumnStatEvent(ColumnStatistics statsObj, List<String> partVals, IHMSHandler handler) {
+  public UpdatePartitionColumnStatEvent(ColumnStatistics statsObj, List<String> partVals,
+                                        Table tableObj, IHMSHandler handler) {
     super(true, handler);
     this.partColStats = statsObj;
     this.partVals = partVals;
     this.validWriteIds = null;
     this.writeId = 0;
     this.parameters = null;
+    this.tableObj = tableObj;
   }
 
   public ColumnStatistics getPartColStats() {
@@ -90,4 +96,6 @@ public class UpdatePartitionColumnStatEvent extends ListenerEvent {
   public List<String> getPartVals() {
     return partVals;
   }
+
+  public Table getTableObj() { return tableObj; }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
index e3a91f9..10c6b44 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageBuilder.java
@@ -300,9 +300,9 @@ public class MessageBuilder {
 
   public JSONUpdatePartitionColumnStatMessage buildUpdatePartitionColumnStatMessage(ColumnStatistics colStats,
                                                             List<String> partVals, Map<String, String> parameters,
-                                                            String validWriteIds, long writeId) {
+                                                            Table tableObj, String validWriteIds, long writeId) {
     return new JSONUpdatePartitionColumnStatMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, now(), colStats, partVals,
-            parameters, validWriteIds, writeId);
+            parameters, tableObj, validWriteIds, writeId);
   }
 
   public JSONDeletePartitionColumnStatMessage buildDeletePartitionColumnStatMessage(String dbName, String colName,

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/UpdatePartitionColumnStatMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/UpdatePartitionColumnStatMessage.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/UpdatePartitionColumnStatMessage.java
index d15c6e6..7eb6c07 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/UpdatePartitionColumnStatMessage.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/UpdatePartitionColumnStatMessage.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hive.metastore.messaging;
 
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.Table;
+
 import java.util.List;
 import java.util.Map;
 
@@ -39,4 +41,6 @@ public abstract class UpdatePartitionColumnStatMessage extends EventMessage {
   public abstract Map<String, String> getParameters();
 
   public abstract List<String> getPartVals();
+
+  public abstract Table getTableObject() throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONUpdatePartitionColumnStatMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONUpdatePartitionColumnStatMessage.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONUpdatePartitionColumnStatMessage.java
index 3f20091..1b35df5 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONUpdatePartitionColumnStatMessage.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONUpdatePartitionColumnStatMessage.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore.messaging.json;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.messaging.MessageBuilder;
 import org.apache.hadoop.hive.metastore.messaging.UpdatePartitionColumnStatMessage;
 import org.apache.thrift.TException;
@@ -48,6 +49,9 @@ public class JSONUpdatePartitionColumnStatMessage extends UpdatePartitionColumnS
   @JsonProperty
   private List<String> partVals;
 
+  @JsonProperty
+  private String tableObjJson;
+
   /**
    * Default constructor, needed for Jackson.
    */
@@ -56,7 +60,8 @@ public class JSONUpdatePartitionColumnStatMessage extends UpdatePartitionColumnS
 
   public JSONUpdatePartitionColumnStatMessage(String server, String servicePrincipal, Long timestamp,
                                               ColumnStatistics colStats, List<String> partVals,
-                                              Map<String, String> parameters, String validWriteIds, long writeId) {
+                                              Map<String, String> parameters,
+                                              Table tableObj, String validWriteIds, long writeId) {
     this.timestamp = timestamp;
     this.server = server;
     this.servicePrincipal = servicePrincipal;
@@ -66,6 +71,7 @@ public class JSONUpdatePartitionColumnStatMessage extends UpdatePartitionColumnS
     this.partVals = partVals;
     try {
       this.colStatsJson = MessageBuilder.createTableColumnStatJson(colStats);
+      this.tableObjJson = MessageBuilder.createTableObjJson(tableObj);
     } catch (TException e) {
       throw new IllegalArgumentException("Could not serialize JSONUpdatePartitionColumnStatMessage : ", e);
     }
@@ -122,6 +128,11 @@ public class JSONUpdatePartitionColumnStatMessage extends UpdatePartitionColumnS
   }
 
   @Override
+  public Table getTableObject() throws Exception {
+    return  (Table) MessageBuilder.getTObj(tableObjJson, Table.class);
+  }
+
+  @Override
   public String toString() {
     try {
       return JSONMessageDeserializer.mapper.writeValueAsString(this);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index aafdef9..481abbc 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -1389,8 +1389,18 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   @Override
   public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
-      List<String> part_names) throws NoSuchObjectException, MetaException, TException {
-    List<Partition> parts = client.get_partitions_by_names(db_name, tbl_name, part_names);
+                                              List<String> part_names) throws NoSuchObjectException, MetaException, TException {
+    return getPartitionsByNames(db_name, tbl_name, part_names, false);
+  }
+
+  @Override
+  public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
+                                              List<String> part_names, boolean get_col_stats)
+          throws NoSuchObjectException, MetaException, TException {
+    GetPartitionsByNamesRequest gpbnr = new GetPartitionsByNamesRequest(db_name, tbl_name);
+    gpbnr.setNames(part_names);
+    gpbnr.setGet_col_stats(get_col_stats);
+    List<Partition> parts = client.get_partitions_by_names_req(gpbnr).getPartitions();
     return fastpath ? parts : deepCopyPartitions(filterHook.filterPartitions(parts));
   }
 
@@ -3262,6 +3272,13 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   }
 
   @Override
+  public List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
+                                              List<String> part_names, boolean getColStats)
+          throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public List<Partition> listPartitionsWithAuthInfo(String catName, String dbName, String tableName,
                                                     List<String> partialPvals, int maxParts,
                                                     String userName, List<String> groupNames) throws

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
index 5ba768f..03378ba 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestStats.java
@@ -56,6 +56,7 @@ import org.slf4j.LoggerFactory;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -245,6 +246,27 @@ public class TestStats {
     Assert.assertEquals(partNames.size(), aggr.getPartsFound());
     Assert.assertEquals(colMap.size(), aggr.getColStatsSize());
     aggr.getColStats().forEach(cso -> colMap.get(cso.getColName()).compareAggr(cso));
+
+    // Test column stats obtained through getPartitions call
+    for (int i = 0; i < partNames.size(); i++) {
+      String partName = partNames.get(i);
+      List<Partition> partitions = catName.equals(NO_CAT) ?
+              client.getPartitionsByNames(dbName, tableName, Collections.singletonList(partName),
+                      true) :
+              client.getPartitionsByNames(catName, dbName, tableName,
+                      Collections.singletonList(partName), true);
+      Partition partition = partitions.get(0);
+      compareStatsForOneTableOrPartition(partition.getColStats().getStatsObj(), i, colMap);
+
+      // Also test that we do not get statistics when not requested
+      partitions = catName.equals(NO_CAT) ?
+              client.getPartitionsByNames(dbName, tableName, Collections.singletonList(partName),
+                      true) :
+              client.getPartitionsByNames(catName, dbName, tableName,
+                      Collections.singletonList(partName), true);
+      partition = partitions.get(0);
+      Assert.assertFalse(partition.isSetColStats());
+    }
   }
 
   private void compareStatsForOneTableOrPartition(List<ColumnStatisticsObj> objs,


[04/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 1ae447d..4f5f8eb 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -7442,6 +7442,10 @@ class Partition {
    * @var bool
    */
   public $isStatsCompliant = null;
+  /**
+   * @var \metastore\ColumnStatistics
+   */
+  public $colStats = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -7504,6 +7508,11 @@ class Partition {
           'var' => 'isStatsCompliant',
           'type' => TType::BOOL,
           ),
+        12 => array(
+          'var' => 'colStats',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\ColumnStatistics',
+          ),
         );
     }
     if (is_array($vals)) {
@@ -7540,6 +7549,9 @@ class Partition {
       if (isset($vals['isStatsCompliant'])) {
         $this->isStatsCompliant = $vals['isStatsCompliant'];
       }
+      if (isset($vals['colStats'])) {
+        $this->colStats = $vals['colStats'];
+      }
     }
   }
 
@@ -7664,6 +7676,14 @@ class Partition {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 12:
+          if ($ftype == TType::STRUCT) {
+            $this->colStats = new \metastore\ColumnStatistics();
+            $xfer += $this->colStats->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -7763,6 +7783,14 @@ class Partition {
       $xfer += $output->writeBool($this->isStatsCompliant);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->colStats !== null) {
+      if (!is_object($this->colStats)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('colStats', TType::STRUCT, 12);
+      $xfer += $this->colStats->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -15898,6 +15926,279 @@ class PartitionValuesResponse {
 
 }
 
+class GetPartitionsByNamesRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $db_name = null;
+  /**
+   * @var string
+   */
+  public $tbl_name = null;
+  /**
+   * @var string[]
+   */
+  public $names = null;
+  /**
+   * @var bool
+   */
+  public $get_col_stats = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'db_name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tbl_name',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'names',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        4 => array(
+          'var' => 'get_col_stats',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['db_name'])) {
+        $this->db_name = $vals['db_name'];
+      }
+      if (isset($vals['tbl_name'])) {
+        $this->tbl_name = $vals['tbl_name'];
+      }
+      if (isset($vals['names'])) {
+        $this->names = $vals['names'];
+      }
+      if (isset($vals['get_col_stats'])) {
+        $this->get_col_stats = $vals['get_col_stats'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsByNamesRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->db_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tbl_name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->names = array();
+            $_size488 = 0;
+            $_etype491 = 0;
+            $xfer += $input->readListBegin($_etype491, $_size488);
+            for ($_i492 = 0; $_i492 < $_size488; ++$_i492)
+            {
+              $elem493 = null;
+              $xfer += $input->readString($elem493);
+              $this->names []= $elem493;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->get_col_stats);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetPartitionsByNamesRequest');
+    if ($this->db_name !== null) {
+      $xfer += $output->writeFieldBegin('db_name', TType::STRING, 1);
+      $xfer += $output->writeString($this->db_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tbl_name !== null) {
+      $xfer += $output->writeFieldBegin('tbl_name', TType::STRING, 2);
+      $xfer += $output->writeString($this->tbl_name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->names !== null) {
+      if (!is_array($this->names)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('names', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRING, count($this->names));
+        {
+          foreach ($this->names as $iter494)
+          {
+            $xfer += $output->writeString($iter494);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->get_col_stats !== null) {
+      $xfer += $output->writeFieldBegin('get_col_stats', TType::BOOL, 4);
+      $xfer += $output->writeBool($this->get_col_stats);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetPartitionsByNamesResult {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Partition[]
+   */
+  public $partitions = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'partitions',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\Partition',
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['partitions'])) {
+        $this->partitions = $vals['partitions'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsByNamesResult';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->partitions = array();
+            $_size495 = 0;
+            $_etype498 = 0;
+            $xfer += $input->readListBegin($_etype498, $_size495);
+            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
+            {
+              $elem500 = null;
+              $elem500 = new \metastore\Partition();
+              $xfer += $elem500->read($input);
+              $this->partitions []= $elem500;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetPartitionsByNamesResult');
+    if ($this->partitions !== null) {
+      if (!is_array($this->partitions)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitions', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->partitions));
+        {
+          foreach ($this->partitions as $iter501)
+          {
+            $xfer += $iter501->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ResourceUri {
   static $_TSPEC;
 
@@ -16184,15 +16485,15 @@ class Function {
         case 8:
           if ($ftype == TType::LST) {
             $this->resourceUris = array();
-            $_size488 = 0;
-            $_etype491 = 0;
-            $xfer += $input->readListBegin($_etype491, $_size488);
-            for ($_i492 = 0; $_i492 < $_size488; ++$_i492)
+            $_size502 = 0;
+            $_etype505 = 0;
+            $xfer += $input->readListBegin($_etype505, $_size502);
+            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
             {
-              $elem493 = null;
-              $elem493 = new \metastore\ResourceUri();
-              $xfer += $elem493->read($input);
-              $this->resourceUris []= $elem493;
+              $elem507 = null;
+              $elem507 = new \metastore\ResourceUri();
+              $xfer += $elem507->read($input);
+              $this->resourceUris []= $elem507;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16262,9 +16563,9 @@ class Function {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourceUris));
         {
-          foreach ($this->resourceUris as $iter494)
+          foreach ($this->resourceUris as $iter508)
           {
-            $xfer += $iter494->write($output);
+            $xfer += $iter508->write($output);
           }
         }
         $output->writeListEnd();
@@ -16611,15 +16912,15 @@ class GetOpenTxnsInfoResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size495 = 0;
-            $_etype498 = 0;
-            $xfer += $input->readListBegin($_etype498, $_size495);
-            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
+            $_size509 = 0;
+            $_etype512 = 0;
+            $xfer += $input->readListBegin($_etype512, $_size509);
+            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
             {
-              $elem500 = null;
-              $elem500 = new \metastore\TxnInfo();
-              $xfer += $elem500->read($input);
-              $this->open_txns []= $elem500;
+              $elem514 = null;
+              $elem514 = new \metastore\TxnInfo();
+              $xfer += $elem514->read($input);
+              $this->open_txns []= $elem514;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16652,9 +16953,9 @@ class GetOpenTxnsInfoResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter501)
+          foreach ($this->open_txns as $iter515)
           {
-            $xfer += $iter501->write($output);
+            $xfer += $iter515->write($output);
           }
         }
         $output->writeListEnd();
@@ -16758,14 +17059,14 @@ class GetOpenTxnsResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->open_txns = array();
-            $_size502 = 0;
-            $_etype505 = 0;
-            $xfer += $input->readListBegin($_etype505, $_size502);
-            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
+            $_size516 = 0;
+            $_etype519 = 0;
+            $xfer += $input->readListBegin($_etype519, $_size516);
+            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
             {
-              $elem507 = null;
-              $xfer += $input->readI64($elem507);
-              $this->open_txns []= $elem507;
+              $elem521 = null;
+              $xfer += $input->readI64($elem521);
+              $this->open_txns []= $elem521;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16812,9 +17113,9 @@ class GetOpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->open_txns));
         {
-          foreach ($this->open_txns as $iter508)
+          foreach ($this->open_txns as $iter522)
           {
-            $xfer += $output->writeI64($iter508);
+            $xfer += $output->writeI64($iter522);
           }
         }
         $output->writeListEnd();
@@ -16989,14 +17290,14 @@ class OpenTxnRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->replSrcTxnIds = array();
-            $_size509 = 0;
-            $_etype512 = 0;
-            $xfer += $input->readListBegin($_etype512, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size523 = 0;
+            $_etype526 = 0;
+            $xfer += $input->readListBegin($_etype526, $_size523);
+            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
             {
-              $elem514 = null;
-              $xfer += $input->readI64($elem514);
-              $this->replSrcTxnIds []= $elem514;
+              $elem528 = null;
+              $xfer += $input->readI64($elem528);
+              $this->replSrcTxnIds []= $elem528;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17056,9 +17357,9 @@ class OpenTxnRequest {
       {
         $output->writeListBegin(TType::I64, count($this->replSrcTxnIds));
         {
-          foreach ($this->replSrcTxnIds as $iter515)
+          foreach ($this->replSrcTxnIds as $iter529)
           {
-            $xfer += $output->writeI64($iter515);
+            $xfer += $output->writeI64($iter529);
           }
         }
         $output->writeListEnd();
@@ -17127,14 +17428,14 @@ class OpenTxnsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size516 = 0;
-            $_etype519 = 0;
-            $xfer += $input->readListBegin($_etype519, $_size516);
-            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
+            $_size530 = 0;
+            $_etype533 = 0;
+            $xfer += $input->readListBegin($_etype533, $_size530);
+            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
             {
-              $elem521 = null;
-              $xfer += $input->readI64($elem521);
-              $this->txn_ids []= $elem521;
+              $elem535 = null;
+              $xfer += $input->readI64($elem535);
+              $this->txn_ids []= $elem535;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17162,9 +17463,9 @@ class OpenTxnsResponse {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter522)
+          foreach ($this->txn_ids as $iter536)
           {
-            $xfer += $output->writeI64($iter522);
+            $xfer += $output->writeI64($iter536);
           }
         }
         $output->writeListEnd();
@@ -17326,14 +17627,14 @@ class AbortTxnsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->txn_ids = array();
-            $_size523 = 0;
-            $_etype526 = 0;
-            $xfer += $input->readListBegin($_etype526, $_size523);
-            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            $_size537 = 0;
+            $_etype540 = 0;
+            $xfer += $input->readListBegin($_etype540, $_size537);
+            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
             {
-              $elem528 = null;
-              $xfer += $input->readI64($elem528);
-              $this->txn_ids []= $elem528;
+              $elem542 = null;
+              $xfer += $input->readI64($elem542);
+              $this->txn_ids []= $elem542;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17361,9 +17662,9 @@ class AbortTxnsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txn_ids));
         {
-          foreach ($this->txn_ids as $iter529)
+          foreach ($this->txn_ids as $iter543)
           {
-            $xfer += $output->writeI64($iter529);
+            $xfer += $output->writeI64($iter543);
           }
         }
         $output->writeListEnd();
@@ -17609,15 +17910,15 @@ class CommitTxnRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->writeEventInfos = array();
-            $_size530 = 0;
-            $_etype533 = 0;
-            $xfer += $input->readListBegin($_etype533, $_size530);
-            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
+            $_size544 = 0;
+            $_etype547 = 0;
+            $xfer += $input->readListBegin($_etype547, $_size544);
+            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
             {
-              $elem535 = null;
-              $elem535 = new \metastore\WriteEventInfo();
-              $xfer += $elem535->read($input);
-              $this->writeEventInfos []= $elem535;
+              $elem549 = null;
+              $elem549 = new \metastore\WriteEventInfo();
+              $xfer += $elem549->read($input);
+              $this->writeEventInfos []= $elem549;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17671,9 +17972,9 @@ class CommitTxnRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->writeEventInfos));
         {
-          foreach ($this->writeEventInfos as $iter536)
+          foreach ($this->writeEventInfos as $iter550)
           {
-            $xfer += $iter536->write($output);
+            $xfer += $iter550->write($output);
           }
         }
         $output->writeListEnd();
@@ -17836,14 +18137,14 @@ class ReplLastIdInfo {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionList = array();
-            $_size537 = 0;
-            $_etype540 = 0;
-            $xfer += $input->readListBegin($_etype540, $_size537);
-            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
+            $_size551 = 0;
+            $_etype554 = 0;
+            $xfer += $input->readListBegin($_etype554, $_size551);
+            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
             {
-              $elem542 = null;
-              $xfer += $input->readString($elem542);
-              $this->partitionList []= $elem542;
+              $elem556 = null;
+              $xfer += $input->readString($elem556);
+              $this->partitionList []= $elem556;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17898,9 +18199,9 @@ class ReplLastIdInfo {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionList));
         {
-          foreach ($this->partitionList as $iter543)
+          foreach ($this->partitionList as $iter557)
           {
-            $xfer += $output->writeString($iter543);
+            $xfer += $output->writeString($iter557);
           }
         }
         $output->writeListEnd();
@@ -18272,14 +18573,14 @@ class ReplTblWriteIdStateRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size544 = 0;
-            $_etype547 = 0;
-            $xfer += $input->readListBegin($_etype547, $_size544);
-            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
+            $_size558 = 0;
+            $_etype561 = 0;
+            $xfer += $input->readListBegin($_etype561, $_size558);
+            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
             {
-              $elem549 = null;
-              $xfer += $input->readString($elem549);
-              $this->partNames []= $elem549;
+              $elem563 = null;
+              $xfer += $input->readString($elem563);
+              $this->partNames []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18332,9 +18633,9 @@ class ReplTblWriteIdStateRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter550)
+          foreach ($this->partNames as $iter564)
           {
-            $xfer += $output->writeString($iter550);
+            $xfer += $output->writeString($iter564);
           }
         }
         $output->writeListEnd();
@@ -18420,14 +18721,14 @@ class GetValidWriteIdsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fullTableNames = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size565 = 0;
+            $_etype568 = 0;
+            $xfer += $input->readListBegin($_etype568, $_size565);
+            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
             {
-              $elem556 = null;
-              $xfer += $input->readString($elem556);
-              $this->fullTableNames []= $elem556;
+              $elem570 = null;
+              $xfer += $input->readString($elem570);
+              $this->fullTableNames []= $elem570;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18469,9 +18770,9 @@ class GetValidWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->fullTableNames));
         {
-          foreach ($this->fullTableNames as $iter557)
+          foreach ($this->fullTableNames as $iter571)
           {
-            $xfer += $output->writeString($iter557);
+            $xfer += $output->writeString($iter571);
           }
         }
         $output->writeListEnd();
@@ -18603,14 +18904,14 @@ class TableValidWriteIds {
         case 3:
           if ($ftype == TType::LST) {
             $this->invalidWriteIds = array();
-            $_size558 = 0;
-            $_etype561 = 0;
-            $xfer += $input->readListBegin($_etype561, $_size558);
-            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
+            $_size572 = 0;
+            $_etype575 = 0;
+            $xfer += $input->readListBegin($_etype575, $_size572);
+            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
             {
-              $elem563 = null;
-              $xfer += $input->readI64($elem563);
-              $this->invalidWriteIds []= $elem563;
+              $elem577 = null;
+              $xfer += $input->readI64($elem577);
+              $this->invalidWriteIds []= $elem577;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18662,9 +18963,9 @@ class TableValidWriteIds {
       {
         $output->writeListBegin(TType::I64, count($this->invalidWriteIds));
         {
-          foreach ($this->invalidWriteIds as $iter564)
+          foreach ($this->invalidWriteIds as $iter578)
           {
-            $xfer += $output->writeI64($iter564);
+            $xfer += $output->writeI64($iter578);
           }
         }
         $output->writeListEnd();
@@ -18739,15 +19040,15 @@ class GetValidWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->tblValidWriteIds = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size579 = 0;
+            $_etype582 = 0;
+            $xfer += $input->readListBegin($_etype582, $_size579);
+            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\TableValidWriteIds();
-              $xfer += $elem570->read($input);
-              $this->tblValidWriteIds []= $elem570;
+              $elem584 = null;
+              $elem584 = new \metastore\TableValidWriteIds();
+              $xfer += $elem584->read($input);
+              $this->tblValidWriteIds []= $elem584;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18775,9 +19076,9 @@ class GetValidWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tblValidWriteIds));
         {
-          foreach ($this->tblValidWriteIds as $iter571)
+          foreach ($this->tblValidWriteIds as $iter585)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter585->write($output);
           }
         }
         $output->writeListEnd();
@@ -18904,14 +19205,14 @@ class AllocateTableWriteIdsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->txnIds = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size586 = 0;
+            $_etype589 = 0;
+            $xfer += $input->readListBegin($_etype589, $_size586);
+            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
             {
-              $elem577 = null;
-              $xfer += $input->readI64($elem577);
-              $this->txnIds []= $elem577;
+              $elem591 = null;
+              $xfer += $input->readI64($elem591);
+              $this->txnIds []= $elem591;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18928,15 +19229,15 @@ class AllocateTableWriteIdsRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->srcTxnToWriteIdList = array();
-            $_size578 = 0;
-            $_etype581 = 0;
-            $xfer += $input->readListBegin($_etype581, $_size578);
-            for ($_i582 = 0; $_i582 < $_size578; ++$_i582)
+            $_size592 = 0;
+            $_etype595 = 0;
+            $xfer += $input->readListBegin($_etype595, $_size592);
+            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
             {
-              $elem583 = null;
-              $elem583 = new \metastore\TxnToWriteId();
-              $xfer += $elem583->read($input);
-              $this->srcTxnToWriteIdList []= $elem583;
+              $elem597 = null;
+              $elem597 = new \metastore\TxnToWriteId();
+              $xfer += $elem597->read($input);
+              $this->srcTxnToWriteIdList []= $elem597;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18974,9 +19275,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txnIds));
         {
-          foreach ($this->txnIds as $iter584)
+          foreach ($this->txnIds as $iter598)
           {
-            $xfer += $output->writeI64($iter584);
+            $xfer += $output->writeI64($iter598);
           }
         }
         $output->writeListEnd();
@@ -18996,9 +19297,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->srcTxnToWriteIdList));
         {
-          foreach ($this->srcTxnToWriteIdList as $iter585)
+          foreach ($this->srcTxnToWriteIdList as $iter599)
           {
-            $xfer += $iter585->write($output);
+            $xfer += $iter599->write($output);
           }
         }
         $output->writeListEnd();
@@ -19161,15 +19462,15 @@ class AllocateTableWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnToWriteIds = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readListBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size600 = 0;
+            $_etype603 = 0;
+            $xfer += $input->readListBegin($_etype603, $_size600);
+            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
             {
-              $elem591 = null;
-              $elem591 = new \metastore\TxnToWriteId();
-              $xfer += $elem591->read($input);
-              $this->txnToWriteIds []= $elem591;
+              $elem605 = null;
+              $elem605 = new \metastore\TxnToWriteId();
+              $xfer += $elem605->read($input);
+              $this->txnToWriteIds []= $elem605;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19197,9 +19498,9 @@ class AllocateTableWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->txnToWriteIds));
         {
-          foreach ($this->txnToWriteIds as $iter592)
+          foreach ($this->txnToWriteIds as $iter606)
           {
-            $xfer += $iter592->write($output);
+            $xfer += $iter606->write($output);
           }
         }
         $output->writeListEnd();
@@ -19544,15 +19845,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size593 = 0;
-            $_etype596 = 0;
-            $xfer += $input->readListBegin($_etype596, $_size593);
-            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
+            $_size607 = 0;
+            $_etype610 = 0;
+            $xfer += $input->readListBegin($_etype610, $_size607);
+            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
             {
-              $elem598 = null;
-              $elem598 = new \metastore\LockComponent();
-              $xfer += $elem598->read($input);
-              $this->component []= $elem598;
+              $elem612 = null;
+              $elem612 = new \metastore\LockComponent();
+              $xfer += $elem612->read($input);
+              $this->component []= $elem612;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19608,9 +19909,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter599)
+          foreach ($this->component as $iter613)
           {
-            $xfer += $iter599->write($output);
+            $xfer += $iter613->write($output);
           }
         }
         $output->writeListEnd();
@@ -20553,15 +20854,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size600 = 0;
-            $_etype603 = 0;
-            $xfer += $input->readListBegin($_etype603, $_size600);
-            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
+            $_size614 = 0;
+            $_etype617 = 0;
+            $xfer += $input->readListBegin($_etype617, $_size614);
+            for ($_i618 = 0; $_i618 < $_size614; ++$_i618)
             {
-              $elem605 = null;
-              $elem605 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem605->read($input);
-              $this->locks []= $elem605;
+              $elem619 = null;
+              $elem619 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem619->read($input);
+              $this->locks []= $elem619;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20589,9 +20890,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter606)
+          foreach ($this->locks as $iter620)
           {
-            $xfer += $iter606->write($output);
+            $xfer += $iter620->write($output);
           }
         }
         $output->writeListEnd();
@@ -20866,17 +21167,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size607 = 0;
-            $_etype610 = 0;
-            $xfer += $input->readSetBegin($_etype610, $_size607);
-            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
+            $_size621 = 0;
+            $_etype624 = 0;
+            $xfer += $input->readSetBegin($_etype624, $_size621);
+            for ($_i625 = 0; $_i625 < $_size621; ++$_i625)
             {
-              $elem612 = null;
-              $xfer += $input->readI64($elem612);
-              if (is_scalar($elem612)) {
-                $this->aborted[$elem612] = true;
+              $elem626 = null;
+              $xfer += $input->readI64($elem626);
+              if (is_scalar($elem626)) {
+                $this->aborted[$elem626] = true;
               } else {
-                $this->aborted []= $elem612;
+                $this->aborted []= $elem626;
               }
             }
             $xfer += $input->readSetEnd();
@@ -20887,17 +21188,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size613 = 0;
-            $_etype616 = 0;
-            $xfer += $input->readSetBegin($_etype616, $_size613);
-            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
+            $_size627 = 0;
+            $_etype630 = 0;
+            $xfer += $input->readSetBegin($_etype630, $_size627);
+            for ($_i631 = 0; $_i631 < $_size627; ++$_i631)
             {
-              $elem618 = null;
-              $xfer += $input->readI64($elem618);
-              if (is_scalar($elem618)) {
-                $this->nosuch[$elem618] = true;
+              $elem632 = null;
+              $xfer += $input->readI64($elem632);
+              if (is_scalar($elem632)) {
+                $this->nosuch[$elem632] = true;
               } else {
-                $this->nosuch []= $elem618;
+                $this->nosuch []= $elem632;
               }
             }
             $xfer += $input->readSetEnd();
@@ -20926,12 +21227,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter619 => $iter620)
+          foreach ($this->aborted as $iter633 => $iter634)
           {
-            if (is_scalar($iter620)) {
-            $xfer += $output->writeI64($iter619);
+            if (is_scalar($iter634)) {
+            $xfer += $output->writeI64($iter633);
             } else {
-            $xfer += $output->writeI64($iter620);
+            $xfer += $output->writeI64($iter634);
             }
           }
         }
@@ -20947,12 +21248,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter621 => $iter622)
+          foreach ($this->nosuch as $iter635 => $iter636)
           {
-            if (is_scalar($iter622)) {
-            $xfer += $output->writeI64($iter621);
+            if (is_scalar($iter636)) {
+            $xfer += $output->writeI64($iter635);
             } else {
-            $xfer += $output->writeI64($iter622);
+            $xfer += $output->writeI64($iter636);
             }
           }
         }
@@ -21111,17 +21412,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size623 = 0;
-            $_ktype624 = 0;
-            $_vtype625 = 0;
-            $xfer += $input->readMapBegin($_ktype624, $_vtype625, $_size623);
-            for ($_i627 = 0; $_i627 < $_size623; ++$_i627)
+            $_size637 = 0;
+            $_ktype638 = 0;
+            $_vtype639 = 0;
+            $xfer += $input->readMapBegin($_ktype638, $_vtype639, $_size637);
+            for ($_i641 = 0; $_i641 < $_size637; ++$_i641)
             {
-              $key628 = '';
-              $val629 = '';
-              $xfer += $input->readString($key628);
-              $xfer += $input->readString($val629);
-              $this->properties[$key628] = $val629;
+              $key642 = '';
+              $val643 = '';
+              $xfer += $input->readString($key642);
+              $xfer += $input->readString($val643);
+              $this->properties[$key642] = $val643;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21174,10 +21475,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter630 => $viter631)
+          foreach ($this->properties as $kiter644 => $viter645)
           {
-            $xfer += $output->writeString($kiter630);
-            $xfer += $output->writeString($viter631);
+            $xfer += $output->writeString($kiter644);
+            $xfer += $output->writeString($viter645);
           }
         }
         $output->writeMapEnd();
@@ -22172,15 +22473,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size646 = 0;
+            $_etype649 = 0;
+            $xfer += $input->readListBegin($_etype649, $_size646);
+            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
             {
-              $elem637 = null;
-              $elem637 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem637->read($input);
-              $this->compacts []= $elem637;
+              $elem651 = null;
+              $elem651 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem651->read($input);
+              $this->compacts []= $elem651;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22208,9 +22509,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter638)
+          foreach ($this->compacts as $iter652)
           {
-            $xfer += $iter638->write($output);
+            $xfer += $iter652->write($output);
           }
         }
         $output->writeListEnd();
@@ -22357,14 +22658,14 @@ class AddDynamicPartitions {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem644 = null;
-              $xfer += $input->readString($elem644);
-              $this->partitionnames []= $elem644;
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->partitionnames []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22419,9 +22720,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter645)
+          foreach ($this->partitionnames as $iter659)
           {
-            $xfer += $output->writeString($iter645);
+            $xfer += $output->writeString($iter659);
           }
         }
         $output->writeListEnd();
@@ -22756,17 +23057,17 @@ class CreationMetadata {
         case 4:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readSetBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readSetBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              if (is_scalar($elem651)) {
-                $this->tablesUsed[$elem651] = true;
+              $elem665 = null;
+              $xfer += $input->readString($elem665);
+              if (is_scalar($elem665)) {
+                $this->tablesUsed[$elem665] = true;
               } else {
-                $this->tablesUsed []= $elem651;
+                $this->tablesUsed []= $elem665;
               }
             }
             $xfer += $input->readSetEnd();
@@ -22824,12 +23125,12 @@ class CreationMetadata {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter652 => $iter653)
+          foreach ($this->tablesUsed as $iter666 => $iter667)
           {
-            if (is_scalar($iter653)) {
-            $xfer += $output->writeString($iter652);
+            if (is_scalar($iter667)) {
+            $xfer += $output->writeString($iter666);
             } else {
-            $xfer += $output->writeString($iter653);
+            $xfer += $output->writeString($iter667);
             }
           }
         }
@@ -22940,14 +23241,14 @@ class NotificationEventRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->eventTypeSkipList = array();
-            $_size654 = 0;
-            $_etype657 = 0;
-            $xfer += $input->readListBegin($_etype657, $_size654);
-            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
+            $_size668 = 0;
+            $_etype671 = 0;
+            $xfer += $input->readListBegin($_etype671, $_size668);
+            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
             {
-              $elem659 = null;
-              $xfer += $input->readString($elem659);
-              $this->eventTypeSkipList []= $elem659;
+              $elem673 = null;
+              $xfer += $input->readString($elem673);
+              $this->eventTypeSkipList []= $elem673;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22985,9 +23286,9 @@ class NotificationEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->eventTypeSkipList));
         {
-          foreach ($this->eventTypeSkipList as $iter660)
+          foreach ($this->eventTypeSkipList as $iter674)
           {
-            $xfer += $output->writeString($iter660);
+            $xfer += $output->writeString($iter674);
           }
         }
         $output->writeListEnd();
@@ -23288,15 +23589,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size661 = 0;
-            $_etype664 = 0;
-            $xfer += $input->readListBegin($_etype664, $_size661);
-            for ($_i665 = 0; $_i665 < $_size661; ++$_i665)
+            $_size675 = 0;
+            $_etype678 = 0;
+            $xfer += $input->readListBegin($_etype678, $_size675);
+            for ($_i679 = 0; $_i679 < $_size675; ++$_i679)
             {
-              $elem666 = null;
-              $elem666 = new \metastore\NotificationEvent();
-              $xfer += $elem666->read($input);
-              $this->events []= $elem666;
+              $elem680 = null;
+              $elem680 = new \metastore\NotificationEvent();
+              $xfer += $elem680->read($input);
+              $this->events []= $elem680;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23324,9 +23625,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter667)
+          foreach ($this->events as $iter681)
           {
-            $xfer += $iter667->write($output);
+            $xfer += $iter681->write($output);
           }
         }
         $output->writeListEnd();
@@ -23755,14 +24056,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size668 = 0;
-            $_etype671 = 0;
-            $xfer += $input->readListBegin($_etype671, $_size668);
-            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
+            $_size682 = 0;
+            $_etype685 = 0;
+            $xfer += $input->readListBegin($_etype685, $_size682);
+            for ($_i686 = 0; $_i686 < $_size682; ++$_i686)
             {
-              $elem673 = null;
-              $xfer += $input->readString($elem673);
-              $this->filesAdded []= $elem673;
+              $elem687 = null;
+              $xfer += $input->readString($elem687);
+              $this->filesAdded []= $elem687;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23772,14 +24073,14 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = array();
-            $_size674 = 0;
-            $_etype677 = 0;
-            $xfer += $input->readListBegin($_etype677, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size688 = 0;
+            $_etype691 = 0;
+            $xfer += $input->readListBegin($_etype691, $_size688);
+            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
             {
-              $elem679 = null;
-              $xfer += $input->readString($elem679);
-              $this->filesAddedChecksum []= $elem679;
+              $elem693 = null;
+              $xfer += $input->readString($elem693);
+              $this->filesAddedChecksum []= $elem693;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23789,14 +24090,14 @@ class InsertEventRequestData {
         case 4:
           if ($ftype == TType::LST) {
             $this->subDirectoryList = array();
-            $_size680 = 0;
-            $_etype683 = 0;
-            $xfer += $input->readListBegin($_etype683, $_size680);
-            for ($_i684 = 0; $_i684 < $_size680; ++$_i684)
+            $_size694 = 0;
+            $_etype697 = 0;
+            $xfer += $input->readListBegin($_etype697, $_size694);
+            for ($_i698 = 0; $_i698 < $_size694; ++$_i698)
             {
-              $elem685 = null;
-              $xfer += $input->readString($elem685);
-              $this->subDirectoryList []= $elem685;
+              $elem699 = null;
+              $xfer += $input->readString($elem699);
+              $this->subDirectoryList []= $elem699;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23829,9 +24130,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter686)
+          foreach ($this->filesAdded as $iter700)
           {
-            $xfer += $output->writeString($iter686);
+            $xfer += $output->writeString($iter700);
           }
         }
         $output->writeListEnd();
@@ -23846,9 +24147,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter687)
+          foreach ($this->filesAddedChecksum as $iter701)
           {
-            $xfer += $output->writeString($iter687);
+            $xfer += $output->writeString($iter701);
           }
         }
         $output->writeListEnd();
@@ -23863,9 +24164,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->subDirectoryList));
         {
-          foreach ($this->subDirectoryList as $iter688)
+          foreach ($this->subDirectoryList as $iter702)
           {
-            $xfer += $output->writeString($iter688);
+            $xfer += $output->writeString($iter702);
           }
         }
         $output->writeListEnd();
@@ -24094,14 +24395,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size689 = 0;
-            $_etype692 = 0;
-            $xfer += $input->readListBegin($_etype692, $_size689);
-            for ($_i693 = 0; $_i693 < $_size689; ++$_i693)
+            $_size703 = 0;
+            $_etype706 = 0;
+            $xfer += $input->readListBegin($_etype706, $_size703);
+            for ($_i707 = 0; $_i707 < $_size703; ++$_i707)
             {
-              $elem694 = null;
-              $xfer += $input->readString($elem694);
-              $this->partitionVals []= $elem694;
+              $elem708 = null;
+              $xfer += $input->readString($elem708);
+              $this->partitionVals []= $elem708;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24159,9 +24460,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter695)
+          foreach ($this->partitionVals as $iter709)
           {
-            $xfer += $output->writeString($iter695);
+            $xfer += $output->writeString($iter709);
           }
         }
         $output->writeListEnd();
@@ -24372,14 +24673,14 @@ class WriteNotificationLogRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size696 = 0;
-            $_etype699 = 0;
-            $xfer += $input->readListBegin($_etype699, $_size696);
-            for ($_i700 = 0; $_i700 < $_size696; ++$_i700)
+            $_size710 = 0;
+            $_etype713 = 0;
+            $xfer += $input->readListBegin($_etype713, $_size710);
+            for ($_i714 = 0; $_i714 < $_size710; ++$_i714)
             {
-              $elem701 = null;
-              $xfer += $input->readString($elem701);
-              $this->partitionVals []= $elem701;
+              $elem715 = null;
+              $xfer += $input->readString($elem715);
+              $this->partitionVals []= $elem715;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24435,9 +24736,9 @@ class WriteNotificationLogRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter702)
+          foreach ($this->partitionVals as $iter716)
           {
-            $xfer += $output->writeString($iter702);
+            $xfer += $output->writeString($iter716);
           }
         }
         $output->writeListEnd();
@@ -24665,18 +24966,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size703 = 0;
-            $_ktype704 = 0;
-            $_vtype705 = 0;
-            $xfer += $input->readMapBegin($_ktype704, $_vtype705, $_size703);
-            for ($_i707 = 0; $_i707 < $_size703; ++$_i707)
+            $_size717 = 0;
+            $_ktype718 = 0;
+            $_vtype719 = 0;
+            $xfer += $input->readMapBegin($_ktype718, $_vtype719, $_size717);
+            for ($_i721 = 0; $_i721 < $_size717; ++$_i721)
             {
-              $key708 = 0;
-              $val709 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key708);
-              $val709 = new \metastore\MetadataPpdResult();
-              $xfer += $val709->read($input);
-              $this->metadata[$key708] = $val709;
+              $key722 = 0;
+              $val723 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key722);
+              $val723 = new \metastore\MetadataPpdResult();
+              $xfer += $val723->read($input);
+              $this->metadata[$key722] = $val723;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -24711,10 +25012,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter710 => $viter711)
+          foreach ($this->metadata as $kiter724 => $viter725)
           {
-            $xfer += $output->writeI64($kiter710);
-            $xfer += $viter711->write($output);
+            $xfer += $output->writeI64($kiter724);
+            $xfer += $viter725->write($output);
           }
         }
         $output->writeMapEnd();
@@ -24816,14 +25117,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size712 = 0;
-            $_etype715 = 0;
-            $xfer += $input->readListBegin($_etype715, $_size712);
-            for ($_i716 = 0; $_i716 < $_size712; ++$_i716)
+            $_size726 = 0;
+            $_etype729 = 0;
+            $xfer += $input->readListBegin($_etype729, $_size726);
+            for ($_i730 = 0; $_i730 < $_size726; ++$_i730)
             {
-              $elem717 = null;
-              $xfer += $input->readI64($elem717);
-              $this->fileIds []= $elem717;
+              $elem731 = null;
+              $xfer += $input->readI64($elem731);
+              $this->fileIds []= $elem731;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24872,9 +25173,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter718)
+          foreach ($this->fileIds as $iter732)
           {
-            $xfer += $output->writeI64($iter718);
+            $xfer += $output->writeI64($iter732);
           }
         }
         $output->writeListEnd();
@@ -24968,17 +25269,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size719 = 0;
-            $_ktype720 = 0;
-            $_vtype721 = 0;
-            $xfer += $input->readMapBegin($_ktype720, $_vtype721, $_size719);
-            for ($_i723 = 0; $_i723 < $_size719; ++$_i723)
+            $_size733 = 0;
+            $_ktype734 = 0;
+            $_vtype735 = 0;
+            $xfer += $input->readMapBegin($_ktype734, $_vtype735, $_size733);
+            for ($_i737 = 0; $_i737 < $_size733; ++$_i737)
             {
-              $key724 = 0;
-              $val725 = '';
-              $xfer += $input->readI64($key724);
-              $xfer += $input->readString($val725);
-              $this->metadata[$key724] = $val725;
+              $key738 = 0;
+              $val739 = '';
+              $xfer += $input->readI64($key738);
+              $xfer += $input->readString($val739);
+              $this->metadata[$key738] = $val739;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25013,10 +25314,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter726 => $viter727)
+          foreach ($this->metadata as $kiter740 => $viter741)
           {
-            $xfer += $output->writeI64($kiter726);
-            $xfer += $output->writeString($viter727);
+            $xfer += $output->writeI64($kiter740);
+            $xfer += $output->writeString($viter741);
           }
         }
         $output->writeMapEnd();
@@ -25085,14 +25386,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size728 = 0;
-            $_etype731 = 0;
-            $xfer += $input->readListBegin($_etype731, $_size728);
-            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
+            $_size742 = 0;
+            $_etype745 = 0;
+            $xfer += $input->readListBegin($_etype745, $_size742);
+            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
             {
-              $elem733 = null;
-              $xfer += $input->readI64($elem733);
-              $this->fileIds []= $elem733;
+              $elem747 = null;
+              $xfer += $input->readI64($elem747);
+              $this->fileIds []= $elem747;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25120,9 +25421,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter734)
+          foreach ($this->fileIds as $iter748)
           {
-            $xfer += $output->writeI64($iter734);
+            $xfer += $output->writeI64($iter748);
           }
         }
         $output->writeListEnd();
@@ -25262,14 +25563,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size735 = 0;
-            $_etype738 = 0;
-            $xfer += $input->readListBegin($_etype738, $_size735);
-            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
+            $_size749 = 0;
+            $_etype752 = 0;
+            $xfer += $input->readListBegin($_etype752, $_size749);
+            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
             {
-              $elem740 = null;
-              $xfer += $input->readI64($elem740);
-              $this->fileIds []= $elem740;
+              $elem754 = null;
+              $xfer += $input->readI64($elem754);
+              $this->fileIds []= $elem754;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25279,14 +25580,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size741 = 0;
-            $_etype744 = 0;
-            $xfer += $input->readListBegin($_etype744, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size755 = 0;
+            $_etype758 = 0;
+            $xfer += $input->readListBegin($_etype758, $_size755);
+            for ($_i759 = 0; $_i759 < $_size755; ++$_i759)
             {
-              $elem746 = null;
-              $xfer += $input->readString($elem746);
-              $this->metadata []= $elem746;
+              $elem760 = null;
+              $xfer += $input->readString($elem760);
+              $this->metadata []= $elem760;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25321,9 +25622,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter747)
+          foreach ($this->fileIds as $iter761)
           {
-            $xfer += $output->writeI64($iter747);
+            $xfer += $output->writeI64($iter761);
           }
         }
         $output->writeListEnd();
@@ -25338,9 +25639,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter748)
+          foreach ($this->metadata as $iter762)
           {
-            $xfer += $output->writeString($iter748);
+            $xfer += $output->writeString($iter762);
           }
         }
         $output->writeListEnd();
@@ -25459,14 +25760,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size749 = 0;
-            $_etype752 = 0;
-            $xfer += $input->readListBegin($_etype752, $_size749);
-            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
+            $_size763 = 0;
+            $_etype766 = 0;
+            $xfer += $input->readListBegin($_etype766, $_size763);
+            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
             {
-              $elem754 = null;
-              $xfer += $input->readI64($elem754);
-              $this->fileIds []= $elem754;
+              $elem768 = null;
+              $xfer += $input->readI64($elem768);
+              $this->fileIds []= $elem768;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25494,9 +25795,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter755)
+          foreach ($this->fileIds as $iter769)
           {
-            $xfer += $output->writeI64($iter755);
+            $xfer += $output->writeI64($iter769);
           }
         }
         $output->writeListEnd();
@@ -25780,15 +26081,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size756 = 0;
-            $_etype759 = 0;
-            $xfer += $input->readListBegin($_etype759, $_size756);
-            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
+            $_size770 = 0;
+            $_etype773 = 0;
+            $xfer += $input->readListBegin($_etype773, $_size770);
+            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
             {
-              $elem761 = null;
-              $elem761 = new \metastore\Function();
-              $xfer += $elem761->read($input);
-              $this->functions []= $elem761;
+              $elem775 = null;
+              $elem775 = new \metastore\Function();
+              $xfer += $elem775->read($input);
+              $this->functions []= $elem775;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25816,9 +26117,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter762)
+          foreach ($this->functions as $iter776)
           {
-            $xfer += $iter762->write($output);
+            $xfer += $iter776->write($output);
           }
         }
         $output->writeListEnd();
@@ -25882,14 +26183,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size763 = 0;
-            $_etype766 = 0;
-            $xfer += $input->readListBegin($_etype766, $_size763);
-            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
+            $_size777 = 0;
+            $_etype780 = 0;
+            $xfer += $input->readListBegin($_etype780, $_size777);
+            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
             {
-              $elem768 = null;
-              $xfer += $input->readI32($elem768);
-              $this->values []= $elem768;
+              $elem782 = null;
+              $xfer += $input->readI32($elem782);
+              $this->values []= $elem782;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25917,9 +26218,9 @@ class ClientCapabilities {
       {
         $output->writeListBegin(TType::I32, count($this->values));
         {
-          foreach ($this->values as $iter769)
+          foreach ($this->values as $iter783)
           {
-            $xfer += $output->writeI32($iter769);
+            $xfer += $output->writeI32($iter783);
           }
         }
         $output->writeListEnd();
@@ -26322,14 +26623,14 @@ class GetTablesRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->tblNames = array();
-            $_size770 = 0;
-            $_etype773 = 0;
-            $xfer += $input->readListBegin($_etype773, $_size770);
-            for ($_i774 = 0; $_i774 < $_size770; ++$_i774)
+            $_size784 = 0;
+            $_etype787 = 0;
+            $xfer += $input->readListBegin($_etype787, $_size784);
+            for ($_i788 = 0; $_i788 < $_size784; ++$_i788)
             {
-              $elem775 = null;
-              $xfer += $input->readString($elem775);
-              $this->tblNames []= $elem775;
+              $elem789 = null;
+              $xfer += $input->readString($elem789);
+              $this->tblNames []= $elem789;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26377,9 +26678,9 @@ class GetTablesRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->tblNames));
         {
-          foreach ($this->tblNames as $iter776)
+          foreach ($this->tblNames as $iter790)
           {
-            $xfer += $output->writeString($iter776);
+            $xfer += $output->writeString($iter790);
           }
         }
         $output->writeListEnd();
@@ -26457,15 +26758,15 @@ class GetTablesResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tables = array();
-            $_size777 = 0;
-            $_etype780 = 0;
-            $xfer += $input->readListBegin($_etype780, $_size777);
-            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
+            $_size791 = 0;
+            $_etype794 = 0;
+            $xfer += $input->readListBegin($_etype794, $_size791);
+            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
             {
-              $elem782 = null;
-              $elem782 = new \metastore\Table();
-              $xfer += $elem782->read($input);
-              $this->tables []= $elem782;
+              $elem796 = null;
+              $elem796 = new \metastore\Table();
+              $xfer += $elem796->read($input);
+              $this->tables []= $elem796;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26493,9 +26794,9 @@ class GetTablesResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tables));
         {
-          foreach ($this->tables as $iter783)
+          foreach ($this->tables as $iter797)
           {
-            $xfer += $iter783->write($output);
+            $xfer += $iter797->write($output);
           }
         }
         $output->writeListEnd();
@@ -28302,15 +28603,15 @@ class WMFullResourcePlan {
         case 2:
           if ($ftype == TType::LST) {
             $this->pools = array();
-            $_size784 = 0;
-            $_etype787 = 0;
-            $xfer += $input->readListBegin($_etype787, $_size784);
-            for ($_i788 = 0; $_i788 < $_size784; ++$_i788)
+            $_size798 = 0;
+            $_etype801 = 0;
+            $xfer += $input->readListBegin($_etype801, $_size798);
+            for ($_i802 = 0; $_i802 < $_size798; ++$_i802)
             {
-              $elem789 = null;
-              $elem789 = new \metastore\WMPool();
-              $xfer += $elem789->read($input);
-              $this->pools []= $elem789;
+              $elem803 = null;
+              $elem803 = new \metastore\WMPool();
+              $xfer += $elem803->read($input);
+              $this->pools []= $elem803;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28320,15 +28621,15 @@ class WMFullResourcePlan {
         case 3:
           if ($ftype == TType::LST) {
             $this->mappings = array();
-            $_size790 = 0;
-            $_etype793 = 0;
-            $xfer += $input->readListBegin($_etype793, $_size790);
-            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
+            $_size804 = 0;
+            $_etype807 = 0;
+            $xfer += $input->readListBegin($_etype807, $_size804);
+            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
             {
-              $elem795 = null;
-              $elem795 = new \metastore\WMMapping();
-              $xfer += $elem795->read($input);
-              $this->mappings []= $elem795;
+              $elem809 = null;
+              $elem809 = new \metastore\WMMapping();
+              $xfer += $elem809->read($input);
+              $this->mappings []= $elem809;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28338,15 +28639,15 @@ class WMFullResourcePlan {
         case 4:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size796 = 0;
-            $_etype799 = 0;
-            $xfer += $input->readListBegin($_etype799, $_size796);
-            for ($_i800 = 0; $_i800 < $_size796; ++$_i800)
+            $_size810 = 0;
+            $_etype813 = 0;
+            $xfer += $input->readListBegin($_etype813, $_size810);
+            for ($_i814 = 0; $_i814 < $_size810; ++$_i814)
             {
-              $elem801 = null;
-              $elem801 = new \metastore\WMTrigger();
-              $xfer += $elem801->read($input);
-              $this->triggers []= $elem801;
+              $elem815 = null;
+              $elem815 = new \metastore\WMTrigger();
+              $xfer += $elem815->read($input);
+              $this->triggers []= $elem815;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28356,15 +28657,15 @@ class WMFullResourcePlan {
         case 5:
           if ($ftype == TType::LST) {
             $this->poolTriggers = array();
-            $_size802 = 0;
-            $_etype805 = 0;
-            $xfer += $input->readListBegin($_etype805, $_size802);
-            for ($_i806 = 0; $_i806 < $_size802; ++$_i806)
+            $_size816 = 0;
+            $_etype819 = 0;
+            $xfer += $input->readListBegin($_etype819, $_size816);
+            for ($_i820 = 0; $_i820 < $_size816; ++$_i820)
             {
-              $elem807 = null;
-              $elem807 = new \metastore\WMPoolTrigger();
-              $xfer += $elem807->read($input);
-              $this->poolTriggers []= $elem807;
+              $elem821 = null;
+              $elem821 = new \metastore\WMPoolTrigger();
+              $xfer += $elem821->read($input);
+              $this->poolTriggers []= $elem821;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28400,9 +28701,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->pools));
         {
-          foreach ($this->pools as $iter808)
+          foreach ($this->pools as $iter822)
           {
-            $xfer += $iter808->write($output);
+            $xfer += $iter822->write($output);
           }
         }
         $output->writeListEnd();
@@ -28417,9 +28718,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->mappings));
         {
-          foreach ($this->mappings as $iter809)
+          foreach ($this->mappings as $iter823)
           {
-            $xfer += $iter809->write($output);
+            $xfer += $iter823->write($output);
           }
         }
         $output->writeListEnd();
@@ -28434,9 +28735,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter810)
+          foreach ($this->triggers as $iter824)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter824->write($output);
           }
         }
         $output->writeListEnd();
@@ -28451,9 +28752,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->poolTriggers));
         {
-          foreach ($this->poolTriggers as $iter811)
+          foreach ($this->poolTriggers as $iter825)
           {
-            $xfer += $iter811->write($output);
+            $xfer += $iter825->write($output);
           }
         }
         $output->writeListEnd();
@@ -29079,15 +29380,15 @@ class WMGetAllResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->resourcePlans = array();
-            $_size812 = 0;
-            $_etype815 = 0;
-            $xfer += $input->readListBegin($_etype815, $_size812);
-            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
+            $_size826 = 0;
+            $_etype829 = 0;
+            $xfer += $input->readListBegin($_etype829, $_size826);
+            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
             {
-              $elem817 = null;
-              $elem817 = new \metastore\WMResourcePlan();
-              $xfer += $elem817->read($input);
-              $this->resourcePlans []= $elem817;
+              $elem831 = null;
+              $elem831 = new \metastore\WMResourcePlan();
+              $xfer += $elem831->read($input);
+              $this->resourcePlans []= $elem831;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29115,9 +29416,9 @@ class WMGetAllResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourcePlans));
         {
-          foreach ($this->resourcePlans as $iter818)
+          foreach ($this->resourcePlans as $iter832)
           {
-            $xfer += $iter818->write($output);
+            $xfer += $iter832->write($output);
           }
         }
         $output->writeListEnd();
@@ -29569,14 +29870,14 @@ class WMValidateResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->errors = array();
-            $_size819 = 0;
-            $_etype822 = 0;
-            $xfer += $input->readListBegin($_etype822, $_size819);
-            for ($_i823 = 0; $_i823 < $_size819; ++$_i823)
+            $_size833 = 0;
+            $_etype836 = 0;
+            $xfer += $input->readListBegin($_etype836, $_size833);
+            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
             {
-              $elem824 = null;
-              $xfer += $input->readString($elem824);
-              $this->errors []= $elem824;
+              $elem838 = null;
+              $xfer += $input->readString($elem838);
+              $this->errors []= $elem838;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29586,14 +29887,14 @@ class WMValidateResourcePlanResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->warnings = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size839 = 0;
+            $_etype842 = 0;
+            $xfer += $input->readListBegin($_etype842, $_size839);
+            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
             {
-              $elem830 = null;
-              $xfer += $input->readString($elem830);
-              $this->warnings []= $elem830;
+              $elem844 = null;
+              $xfer += $input->readString($elem844);
+              $this->warnings []= $elem844;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29621,9 +29922,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter831)
+          foreach ($this->errors as $iter845)
           {
-            $xfer += $output->writeString($iter831);
+            $xfer += $output->writeString($iter845);
           }
         }
         $output->writeListEnd();
@@ -29638,9 +29939,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->warnings));
         {
-          foreach ($this->warnings as $iter832)
+          foreach ($this->warnings as $iter846)
           {
-            $xfer += $output->writeString($iter832);
+            $xfer += $output->writeString($iter846);
           }
         }
         $output->writeListEnd();
@@ -30382,15 +30683,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size833 = 0;
-            $_etype836 = 0;
-            $xfer += $input->readListBegin($_etype836, $_size833);
-            for ($_i837 = 0; $_i837 < $_size833; ++$_i837)
+            $_size847 = 0;
+            $_etype850 = 0;
+            $xfer += $input->readListBegin($_etype850, $_size847);
+            for ($_i851 = 0; $_i851 < $_size847; ++$_i851)
             {
-              $elem838 = null;
-              $elem838 = new \metastore\WMTrigger();
-              $xfer += $elem838->read($input);
-              $this->triggers []= $elem838;
+              $elem852 = null;
+              $elem852 = new \metastore\WMTrigger();
+              $xfer += $elem852->read($input);
+              $this->triggers []= $elem852;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30418,9 +30719,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter839)
+          foreach ($this->triggers as $iter853)
           {
-            $xfer += $iter839->write($output);
+            $xfer += $iter853->write($output);
           }
         }
         $output->writeListEnd();
@@ -32050,15 +32351,15 @@ class SchemaVersion {
         case 4:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size840 = 0;
-            $_etype843 = 0;
-            $xfer += $input->readListBegin($_etype843, $_size840);
-            for ($_i844 = 0; $_i844 < $_size840; ++$_i844)
+            $_size854 = 0;
+            $_etype857 = 0;
+            $xfer += $input->readListBegin($_etype857, $_size854);
+            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
             {
-              $elem845 = null;
-              $elem845 = new \metastore\FieldSchema();
-              $xfer += $elem845->read($input);
-              $this->cols []= $elem845;
+              $elem859 = null;
+              $elem859 = new \metastore\FieldSchema();
+              $xfer += $elem859->read($input);
+              $this->cols []= $elem859;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32147,9 +32448,9 @@ class SchemaVersion {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter846)
+          foreach ($this->cols as $iter860)
           {
-            $xfer += $iter846->write($output);
+            $xfer += $iter860->write($output);
           }
         }
         $output->writeListEnd();
@@ -32471,15 +32772,15 @@ class FindSchemasByColsResp {
         case 1:
           if ($ftype == TType::LST) {
             $this->schemaVersions = array();
-            $_size847 = 0;
-            $_etype850 = 0;
-            $xfer += $input->readListBegin($_etype850, $_size847);
-            for ($_i851 = 0; $_i851 < $_size847; ++$_i851)
+            $_size861 = 0;
+            $_etype864 = 0;
+            $xfer += $input->readListBegin($_etype864, $_size861);
+            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
             {
-              $elem852 = null;
-              $elem852 = new \metastore\SchemaVersionDescriptor();
-              $xfer += $elem852->read($input);
-              $this->schemaVersions []= $elem852;
+              $elem866 = null;
+              $elem866 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem866->read($input);
+              $this->schemaVersions []= $elem866;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32507,9 +32808,9 @@ class FindSchemasByColsResp {
       {
         $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
         {
-          foreach ($this->schemaVersions as $iter853)
+          foreach ($this->schemaVersions as $iter867)
           {
-            $xfer += $iter853->write($output);
+            $xfer += $iter867->write($output);
           }
         }
         $output->writeListEnd();
@@ -33162,15 +33463,15 @@ class AlterPartitionsRequest {
         case 4:
           if ($ftype == TType::LST) {
             $this->partitions = array();
-            $_size854 = 0;
-            $_etype857 = 0;
-            $xfer += $input->readListBegin($_etype857, $_size854);
-            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
+            $_size868 = 0;
+            $_etype871 = 0;
+            $xfer += $input->readListBegin($_etype871, $_size868);
+            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
             {
-              $elem859 = null;
-              $elem859 = new \metastore\Partition();
-              $xfer += $elem859->read($input);
-              $this->partitions []= $elem859;
+              $elem873 = null;
+              $elem873 = new \metastore\Partition();
+              $xfer += $elem873->read($input);
+              $this->partitions []= $elem873;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33235,9 +33536,9 @@ class AlterPartitionsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitions));
         {
-          foreach ($this->partitions as $iter860)
+          foreach ($this->partitions as $iter874)
           {
-            $xfer += $iter860->write($output);
+            $xfer += $iter874->write($output);
           }
         }
         $output->writeListEnd();
@@ -33446,14 +33747,14 @@ class RenamePartitionRequest {
         case 4:
           if ($ftype == TType::LST) {
             $this->partVals = array();
-            $_size861 = 0;
-            $_etype864 = 0;
-            $xfer += $input->readListBegin($_etype864, $_size861);
-            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
+            $_size875 = 0;
+            $_etype878 = 0;
+            $xfer += $input->readListBegin($_etype878, $_size875);
+            for ($_i879 = 0; $_i879 < $_size875; ++$_i879)
             {
-              $elem866 = null;
-              $xfer += $input->readString($elem866);
-              $this->partVals []= $elem866;
+              $elem880 = null;
+              $xfer += $input->readString($elem880);
+              $this->partVals []= $elem880;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33511,9 +33812,9 @@ class RenamePartitionRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partVals));
         {
-          foreach ($this->partVals as $iter867)
+          foreach ($this->partVals as $iter881)
           {
-            $xfer += $output->writeString($iter867);
+            $xfer += $output->writeString($iter881);
           }
         }
         $output->writeListEnd();
@@ -33935,14 +34236,14 @@ class GetPartitionsProjectionSpec {
         case 1:
           if ($ftype == TType::LST) {
             $this->fieldList = array();
-            $_size868 = 0;
-            $_etype871 = 0;
-            $xfer += $input->readListBegin($_etype871, $_size868);
-            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
+            $_size882 = 0;
+            $_etype885 = 0;
+            $xfer += $input->readListBegin($_etype885, $_size882);
+            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
             {
-              $elem873 = null;
-              $xfer += $input->readString($elem873);
-              $this->fieldList []= $elem873;
+              $elem887 = null;
+              $xfer += $input->readString($elem887);
+              $this->fieldList []= $elem887;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33984,9 +34285,9 @@ class GetPartitionsProjectionSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->fieldList));
         {
-          foreach ($this->fieldList as $iter874)
+          foreach ($this->fieldList as $iter888)
           {
-            $xfer += $output->writeString($iter874);
+            $xfer += $output->writeString($iter888);
           }
         }
         $output->writeListEnd();
@@ -34078,14 +34379,14 @@ class GetPartitionsFilterSpec {
         case 8:
           if ($ftype == TType::LST) {
             $this->filters = array();
-            $_size875 = 0;
-            $_etype878 = 0;
-            $xfer += $input->readListBegin($_etype878, $_size875);
-            for ($_i879 = 0; $_i879 < $_size875; ++$_i879)
+            $_size889 = 0;
+            $_etype892 = 0;
+            $xfer += $input->readListBegin($_etype892, $_size889);
+            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
             {
-              $elem880 = null;
-              $xfer += $input->readString($elem880);
-              $this->filters []= $elem880;
+              $elem894 = null;
+              $xfer += $input->readString($elem894);
+              $this->filters []= $elem894;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34118,9 +34419,9 @@ class GetPartitionsFilterSpec {
       {
         $output->writeListBegin(TType::STRING, count($this->filters));
         {
-          foreach ($this->filters as $iter881)
+          foreach ($this->filters as $iter895)
           {
-            $xfer += $output->writeString($iter881);
+            $xfer += $output->writeString($iter895);
           }
         }
         $output->writeListEnd();
@@ -34185,15 +34486,15 @@ class GetPartitionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->partitionSpec = array();
-            $_size882 = 0;
-            $_etype885 = 0;
-            $xfer += $input->readListBegin($_etype885, $_size882);
-            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
+            $_size896 = 0;
+            $_etype899 = 0;
+            $xfer += $input->readListBegin($_etype899, $_size896);
+            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
             {
-              $elem887 = null;
-              $elem887 = new \metastore\PartitionSpec();
-              $xfer += $elem887->read($input);
-              $this->partitionSpec []= $elem887;
+              $elem901 = null;
+              $elem901 = new \metastore\PartitionSpec();
+              $xfer += $elem901->read($input);
+              $this->partitionSpec []= $elem901;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34221,9 +34522,9 @@ class GetPartitionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->partitionSpec));
         {
-          foreach ($this->partitionSpec as $iter888)
+          foreach ($this->partitionSpec as $iter902)
           {
-            $xfer += $iter888->write($output);
+            $xfer += $iter902->write($output);
           }
         }
         $output->writeListEnd();
@@ -34401,14 +34702,14 @@ class GetPartitionsRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->groupNames = array();
-            $_size889 = 0;
-            $_etype892 = 0;
-            $xfer += $input->readListBegin($_etype892, $_size889);
-            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
+            $_size903 = 0;
+            $_etype906 = 0;
+            $xfer += $input->readListBegin($_etype906, $_size903);
+            for ($_i907 = 0; $_i907 < $_size903; ++$_i907)
             {
-              $elem894 = null;
-              $xfer += $input->readString($elem894);
-              $this->groupNames []= $elem894;
+              $elem908 = null;
+              $xfer += $input->readString($elem908);
+              $this->groupNames []= $elem908;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34477,9 +34778,9 @@ class GetPartitionsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->groupNames));
         {
-          foreach ($this->groupNames as $iter895)
+          foreach ($this->groupNames as $iter909)
           {
-            $xfer += $output->writeString($iter895);
+            $xfer += $output->writeString($iter909);
           }
         }
         $output->writeListEnd();


[08/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index 663f7d0..acac747 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list674.size);
-                LockComponent _elem675;
-                for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                org.apache.thrift.protocol.TList _list690 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list690.size);
+                LockComponent _elem691;
+                for (int _i692 = 0; _i692 < _list690.size; ++_i692)
                 {
-                  _elem675 = new LockComponent();
-                  _elem675.read(iprot);
-                  struct.component.add(_elem675);
+                  _elem691 = new LockComponent();
+                  _elem691.read(iprot);
+                  struct.component.add(_elem691);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter677 : struct.component)
+          for (LockComponent _iter693 : struct.component)
           {
-            _iter677.write(oprot);
+            _iter693.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter678 : struct.component)
+        for (LockComponent _iter694 : struct.component)
         {
-          _iter678.write(oprot);
+          _iter694.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list679.size);
-        LockComponent _elem680;
-        for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+        org.apache.thrift.protocol.TList _list695 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list695.size);
+        LockComponent _elem696;
+        for (int _i697 = 0; _i697 < _list695.size; ++_i697)
         {
-          _elem680 = new LockComponent();
-          _elem680.read(iprot);
-          struct.component.add(_elem680);
+          _elem696 = new LockComponent();
+          _elem696.read(iprot);
+          struct.component.add(_elem696);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
index 288c365..d5c19e1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventRequest.java
@@ -525,13 +525,13 @@ import org.slf4j.LoggerFactory;
           case 3: // EVENT_TYPE_SKIP_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                struct.eventTypeSkipList = new ArrayList<String>(_list740.size);
-                String _elem741;
-                for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                struct.eventTypeSkipList = new ArrayList<String>(_list756.size);
+                String _elem757;
+                for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                 {
-                  _elem741 = iprot.readString();
-                  struct.eventTypeSkipList.add(_elem741);
+                  _elem757 = iprot.readString();
+                  struct.eventTypeSkipList.add(_elem757);
                 }
                 iprot.readListEnd();
               }
@@ -566,9 +566,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(EVENT_TYPE_SKIP_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.eventTypeSkipList.size()));
-            for (String _iter743 : struct.eventTypeSkipList)
+            for (String _iter759 : struct.eventTypeSkipList)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -607,9 +607,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetEventTypeSkipList()) {
         {
           oprot.writeI32(struct.eventTypeSkipList.size());
-          for (String _iter744 : struct.eventTypeSkipList)
+          for (String _iter760 : struct.eventTypeSkipList)
           {
-            oprot.writeString(_iter744);
+            oprot.writeString(_iter760);
           }
         }
       }
@@ -627,13 +627,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.eventTypeSkipList = new ArrayList<String>(_list745.size);
-          String _elem746;
-          for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+          org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.eventTypeSkipList = new ArrayList<String>(_list761.size);
+          String _elem762;
+          for (int _i763 = 0; _i763 < _list761.size; ++_i763)
           {
-            _elem746 = iprot.readString();
-            struct.eventTypeSkipList.add(_elem746);
+            _elem762 = iprot.readString();
+            struct.eventTypeSkipList.add(_elem762);
           }
         }
         struct.setEventTypeSkipListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index b86f038..1f2776b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list748.size);
-                NotificationEvent _elem749;
-                for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list764.size);
+                NotificationEvent _elem765;
+                for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                 {
-                  _elem749 = new NotificationEvent();
-                  _elem749.read(iprot);
-                  struct.events.add(_elem749);
+                  _elem765 = new NotificationEvent();
+                  _elem765.read(iprot);
+                  struct.events.add(_elem765);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter751 : struct.events)
+          for (NotificationEvent _iter767 : struct.events)
           {
-            _iter751.write(oprot);
+            _iter767.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter752 : struct.events)
+        for (NotificationEvent _iter768 : struct.events)
         {
-          _iter752.write(oprot);
+          _iter768.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list753.size);
-        NotificationEvent _elem754;
-        for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+        org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list769.size);
+        NotificationEvent _elem770;
+        for (int _i771 = 0; _i771 < _list769.size; ++_i771)
         {
-          _elem754 = new NotificationEvent();
-          _elem754.read(iprot);
-          struct.events.add(_elem754);
+          _elem770 = new NotificationEvent();
+          _elem770.read(iprot);
+          struct.events.add(_elem770);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
index b5efffd..a8fd4b7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnRequest.java
@@ -904,13 +904,13 @@ import org.slf4j.LoggerFactory;
           case 6: // REPL_SRC_TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list578 = iprot.readListBegin();
-                struct.replSrcTxnIds = new ArrayList<Long>(_list578.size);
-                long _elem579;
-                for (int _i580 = 0; _i580 < _list578.size; ++_i580)
+                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
+                struct.replSrcTxnIds = new ArrayList<Long>(_list594.size);
+                long _elem595;
+                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
                 {
-                  _elem579 = iprot.readI64();
-                  struct.replSrcTxnIds.add(_elem579);
+                  _elem595 = iprot.readI64();
+                  struct.replSrcTxnIds.add(_elem595);
                 }
                 iprot.readListEnd();
               }
@@ -972,9 +972,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(REPL_SRC_TXN_IDS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.replSrcTxnIds.size()));
-            for (long _iter581 : struct.replSrcTxnIds)
+            for (long _iter597 : struct.replSrcTxnIds)
             {
-              oprot.writeI64(_iter581);
+              oprot.writeI64(_iter597);
             }
             oprot.writeListEnd();
           }
@@ -1031,9 +1031,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetReplSrcTxnIds()) {
         {
           oprot.writeI32(struct.replSrcTxnIds.size());
-          for (long _iter582 : struct.replSrcTxnIds)
+          for (long _iter598 : struct.replSrcTxnIds)
           {
-            oprot.writeI64(_iter582);
+            oprot.writeI64(_iter598);
           }
         }
       }
@@ -1062,13 +1062,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list583 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.replSrcTxnIds = new ArrayList<Long>(_list583.size);
-          long _elem584;
-          for (int _i585 = 0; _i585 < _list583.size; ++_i585)
+          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.replSrcTxnIds = new ArrayList<Long>(_list599.size);
+          long _elem600;
+          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
           {
-            _elem584 = iprot.readI64();
-            struct.replSrcTxnIds.add(_elem584);
+            _elem600 = iprot.readI64();
+            struct.replSrcTxnIds.add(_elem600);
           }
         }
         struct.setReplSrcTxnIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
index 71a2c4f..f0e6bdb 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/OpenTxnsResponse.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list586 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list586.size);
-                long _elem587;
-                for (int _i588 = 0; _i588 < _list586.size; ++_i588)
+                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list602.size);
+                long _elem603;
+                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
                 {
-                  _elem587 = iprot.readI64();
-                  struct.txn_ids.add(_elem587);
+                  _elem603 = iprot.readI64();
+                  struct.txn_ids.add(_elem603);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter589 : struct.txn_ids)
+          for (long _iter605 : struct.txn_ids)
           {
-            oprot.writeI64(_iter589);
+            oprot.writeI64(_iter605);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter590 : struct.txn_ids)
+        for (long _iter606 : struct.txn_ids)
         {
-          oprot.writeI64(_iter590);
+          oprot.writeI64(_iter606);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, OpenTxnsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list591 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list591.size);
-        long _elem592;
-        for (int _i593 = 0; _i593 < _list591.size; ++_i593)
+        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list607.size);
+        long _elem608;
+        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
         {
-          _elem592 = iprot.readI64();
-          struct.txn_ids.add(_elem592);
+          _elem608 = iprot.readI64();
+          struct.txn_ids.add(_elem608);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index 0156a9a..a191255 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@ -49,6 +49,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
   private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)10);
   private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.BOOL, (short)11);
+  private static final org.apache.thrift.protocol.TField COL_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("colStats", org.apache.thrift.protocol.TType.STRUCT, (short)12);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -67,6 +68,7 @@ import org.slf4j.LoggerFactory;
   private String catName; // optional
   private long writeId; // optional
   private boolean isStatsCompliant; // optional
+  private ColumnStatistics colStats; // 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 {
@@ -80,7 +82,8 @@ import org.slf4j.LoggerFactory;
     PRIVILEGES((short)8, "privileges"),
     CAT_NAME((short)9, "catName"),
     WRITE_ID((short)10, "writeId"),
-    IS_STATS_COMPLIANT((short)11, "isStatsCompliant");
+    IS_STATS_COMPLIANT((short)11, "isStatsCompliant"),
+    COL_STATS((short)12, "colStats");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -117,6 +120,8 @@ import org.slf4j.LoggerFactory;
           return WRITE_ID;
         case 11: // IS_STATS_COMPLIANT
           return IS_STATS_COMPLIANT;
+        case 12: // COL_STATS
+          return COL_STATS;
         default:
           return null;
       }
@@ -162,7 +167,7 @@ import org.slf4j.LoggerFactory;
   private static final int __WRITEID_ISSET_ID = 2;
   private static final int __ISSTATSCOMPLIANT_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.CAT_NAME,_Fields.WRITE_ID,_Fields.IS_STATS_COMPLIANT};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.CAT_NAME,_Fields.WRITE_ID,_Fields.IS_STATS_COMPLIANT,_Fields.COL_STATS};
   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);
@@ -191,6 +196,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.IS_STATS_COMPLIANT, new org.apache.thrift.meta_data.FieldMetaData("isStatsCompliant", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.COL_STATS, new org.apache.thrift.meta_data.FieldMetaData("colStats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "ColumnStatistics")));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Partition.class, metaDataMap);
   }
@@ -253,6 +260,9 @@ import org.slf4j.LoggerFactory;
     }
     this.writeId = other.writeId;
     this.isStatsCompliant = other.isStatsCompliant;
+    if (other.isSetColStats()) {
+      this.colStats = other.colStats;
+    }
   }
 
   public Partition deepCopy() {
@@ -276,6 +286,7 @@ import org.slf4j.LoggerFactory;
 
     setIsStatsCompliantIsSet(false);
     this.isStatsCompliant = false;
+    this.colStats = null;
   }
 
   public int getValuesSize() {
@@ -553,6 +564,29 @@ import org.slf4j.LoggerFactory;
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSTATSCOMPLIANT_ISSET_ID, value);
   }
 
+  public ColumnStatistics getColStats() {
+    return this.colStats;
+  }
+
+  public void setColStats(ColumnStatistics colStats) {
+    this.colStats = colStats;
+  }
+
+  public void unsetColStats() {
+    this.colStats = null;
+  }
+
+  /** Returns true if field colStats is set (has been assigned a value) and false otherwise */
+  public boolean isSetColStats() {
+    return this.colStats != null;
+  }
+
+  public void setColStatsIsSet(boolean value) {
+    if (!value) {
+      this.colStats = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case VALUES:
@@ -643,6 +677,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case COL_STATS:
+      if (value == null) {
+        unsetColStats();
+      } else {
+        setColStats((ColumnStatistics)value);
+      }
+      break;
+
     }
   }
 
@@ -681,6 +723,9 @@ import org.slf4j.LoggerFactory;
     case IS_STATS_COMPLIANT:
       return isIsStatsCompliant();
 
+    case COL_STATS:
+      return getColStats();
+
     }
     throw new IllegalStateException();
   }
@@ -714,6 +759,8 @@ import org.slf4j.LoggerFactory;
       return isSetWriteId();
     case IS_STATS_COMPLIANT:
       return isSetIsStatsCompliant();
+    case COL_STATS:
+      return isSetColStats();
     }
     throw new IllegalStateException();
   }
@@ -830,6 +877,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_colStats = true && this.isSetColStats();
+    boolean that_present_colStats = true && that.isSetColStats();
+    if (this_present_colStats || that_present_colStats) {
+      if (!(this_present_colStats && that_present_colStats))
+        return false;
+      if (!this.colStats.equals(that.colStats))
+        return false;
+    }
+
     return true;
   }
 
@@ -892,6 +948,11 @@ import org.slf4j.LoggerFactory;
     if (present_isStatsCompliant)
       list.add(isStatsCompliant);
 
+    boolean present_colStats = true && (isSetColStats());
+    list.add(present_colStats);
+    if (present_colStats)
+      list.add(colStats);
+
     return list.hashCode();
   }
 
@@ -1013,6 +1074,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetColStats()).compareTo(other.isSetColStats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColStats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colStats, other.colStats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1112,6 +1183,16 @@ import org.slf4j.LoggerFactory;
       sb.append(this.isStatsCompliant);
       first = false;
     }
+    if (isSetColStats()) {
+      if (!first) sb.append(", ");
+      sb.append("colStats:");
+      if (this.colStats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.colStats);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1275,6 +1356,15 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 12: // COL_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.colStats = new ColumnStatistics();
+              struct.colStats.read(iprot);
+              struct.setColStatsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1358,6 +1448,13 @@ import org.slf4j.LoggerFactory;
         oprot.writeBool(struct.isStatsCompliant);
         oprot.writeFieldEnd();
       }
+      if (struct.colStats != null) {
+        if (struct.isSetColStats()) {
+          oprot.writeFieldBegin(COL_STATS_FIELD_DESC);
+          struct.colStats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1409,7 +1506,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetIsStatsCompliant()) {
         optionals.set(10);
       }
-      oprot.writeBitSet(optionals, 11);
+      if (struct.isSetColStats()) {
+        optionals.set(11);
+      }
+      oprot.writeBitSet(optionals, 12);
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
@@ -1456,12 +1556,15 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetIsStatsCompliant()) {
         oprot.writeBool(struct.isStatsCompliant);
       }
+      if (struct.isSetColStats()) {
+        struct.colStats.write(oprot);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Partition struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(11);
+      BitSet incoming = iprot.readBitSet(12);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TList _list235 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
@@ -1528,6 +1631,11 @@ import org.slf4j.LoggerFactory;
         struct.isStatsCompliant = iprot.readBool();
         struct.setIsStatsCompliantIsSet(true);
       }
+      if (incoming.get(11)) {
+        struct.colStats = new ColumnStatistics();
+        struct.colStats.read(iprot);
+        struct.setColStatsIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 5cbfe64..8518e44 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list832 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list832.size);
-                long _elem833;
-                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
+                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list848.size);
+                long _elem849;
+                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
                 {
-                  _elem833 = iprot.readI64();
-                  struct.fileIds.add(_elem833);
+                  _elem849 = iprot.readI64();
+                  struct.fileIds.add(_elem849);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list835 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list835.size);
-                ByteBuffer _elem836;
-                for (int _i837 = 0; _i837 < _list835.size; ++_i837)
+                org.apache.thrift.protocol.TList _list851 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list851.size);
+                ByteBuffer _elem852;
+                for (int _i853 = 0; _i853 < _list851.size; ++_i853)
                 {
-                  _elem836 = iprot.readBinary();
-                  struct.metadata.add(_elem836);
+                  _elem852 = iprot.readBinary();
+                  struct.metadata.add(_elem852);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter838 : struct.fileIds)
+          for (long _iter854 : struct.fileIds)
           {
-            oprot.writeI64(_iter838);
+            oprot.writeI64(_iter854);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter839 : struct.metadata)
+          for (ByteBuffer _iter855 : struct.metadata)
           {
-            oprot.writeBinary(_iter839);
+            oprot.writeBinary(_iter855);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter840 : struct.fileIds)
+        for (long _iter856 : struct.fileIds)
         {
-          oprot.writeI64(_iter840);
+          oprot.writeI64(_iter856);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter841 : struct.metadata)
+        for (ByteBuffer _iter857 : struct.metadata)
         {
-          oprot.writeBinary(_iter841);
+          oprot.writeBinary(_iter857);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list842 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list842.size);
-        long _elem843;
-        for (int _i844 = 0; _i844 < _list842.size; ++_i844)
+        org.apache.thrift.protocol.TList _list858 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list858.size);
+        long _elem859;
+        for (int _i860 = 0; _i860 < _list858.size; ++_i860)
         {
-          _elem843 = iprot.readI64();
-          struct.fileIds.add(_elem843);
+          _elem859 = iprot.readI64();
+          struct.fileIds.add(_elem859);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list845 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list845.size);
-        ByteBuffer _elem846;
-        for (int _i847 = 0; _i847 < _list845.size; ++_i847)
+        org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list861.size);
+        ByteBuffer _elem862;
+        for (int _i863 = 0; _i863 < _list861.size; ++_i863)
         {
-          _elem846 = iprot.readBinary();
-          struct.metadata.add(_elem846);
+          _elem862 = iprot.readBinary();
+          struct.metadata.add(_elem862);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java
index ea4cc16..6851625 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RenamePartitionRequest.java
@@ -796,13 +796,13 @@ import org.slf4j.LoggerFactory;
           case 4: // PART_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list976 = iprot.readListBegin();
-                struct.partVals = new ArrayList<String>(_list976.size);
-                String _elem977;
-                for (int _i978 = 0; _i978 < _list976.size; ++_i978)
+                org.apache.thrift.protocol.TList _list992 = iprot.readListBegin();
+                struct.partVals = new ArrayList<String>(_list992.size);
+                String _elem993;
+                for (int _i994 = 0; _i994 < _list992.size; ++_i994)
                 {
-                  _elem977 = iprot.readString();
-                  struct.partVals.add(_elem977);
+                  _elem993 = iprot.readString();
+                  struct.partVals.add(_elem993);
                 }
                 iprot.readListEnd();
               }
@@ -862,9 +862,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partVals.size()));
-          for (String _iter979 : struct.partVals)
+          for (String _iter995 : struct.partVals)
           {
-            oprot.writeString(_iter979);
+            oprot.writeString(_iter995);
           }
           oprot.writeListEnd();
         }
@@ -903,9 +903,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tableName);
       {
         oprot.writeI32(struct.partVals.size());
-        for (String _iter980 : struct.partVals)
+        for (String _iter996 : struct.partVals)
         {
-          oprot.writeString(_iter980);
+          oprot.writeString(_iter996);
         }
       }
       struct.newPart.write(oprot);
@@ -933,13 +933,13 @@ import org.slf4j.LoggerFactory;
       struct.tableName = iprot.readString();
       struct.setTableNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list981 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partVals = new ArrayList<String>(_list981.size);
-        String _elem982;
-        for (int _i983 = 0; _i983 < _list981.size; ++_i983)
+        org.apache.thrift.protocol.TList _list997 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partVals = new ArrayList<String>(_list997.size);
+        String _elem998;
+        for (int _i999 = 0; _i999 < _list997.size; ++_i999)
         {
-          _elem982 = iprot.readString();
-          struct.partVals.add(_elem982);
+          _elem998 = iprot.readString();
+          struct.partVals.add(_elem998);
         }
       }
       struct.setPartValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplLastIdInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplLastIdInfo.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplLastIdInfo.java
index 4e317d5..2dab332 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplLastIdInfo.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplLastIdInfo.java
@@ -788,13 +788,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.partitionList = new ArrayList<String>(_list610.size);
-                String _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                struct.partitionList = new ArrayList<String>(_list626.size);
+                String _elem627;
+                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                 {
-                  _elem611 = iprot.readString();
-                  struct.partitionList.add(_elem611);
+                  _elem627 = iprot.readString();
+                  struct.partitionList.add(_elem627);
                 }
                 iprot.readListEnd();
               }
@@ -851,9 +851,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionList.size()));
-            for (String _iter613 : struct.partitionList)
+            for (String _iter629 : struct.partitionList)
             {
-              oprot.writeString(_iter613);
+              oprot.writeString(_iter629);
             }
             oprot.writeListEnd();
           }
@@ -907,9 +907,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionList()) {
         {
           oprot.writeI32(struct.partitionList.size());
-          for (String _iter614 : struct.partitionList)
+          for (String _iter630 : struct.partitionList)
           {
-            oprot.writeString(_iter614);
+            oprot.writeString(_iter630);
           }
         }
       }
@@ -936,13 +936,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionList = new ArrayList<String>(_list615.size);
-          String _elem616;
-          for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+          org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionList = new ArrayList<String>(_list631.size);
+          String _elem632;
+          for (int _i633 = 0; _i633 < _list631.size; ++_i633)
           {
-            _elem616 = iprot.readString();
-            struct.partitionList.add(_elem616);
+            _elem632 = iprot.readString();
+            struct.partitionList.add(_elem632);
           }
         }
         struct.setPartitionListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
index 0fa8067..1bfe434 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
@@ -813,13 +813,13 @@ import org.slf4j.LoggerFactory;
           case 6: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list618.size);
-                String _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list634.size);
+                String _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem619 = iprot.readString();
-                  struct.partNames.add(_elem619);
+                  _elem635 = iprot.readString();
+                  struct.partNames.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -871,9 +871,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter621 : struct.partNames)
+            for (String _iter637 : struct.partNames)
             {
-              oprot.writeString(_iter621);
+              oprot.writeString(_iter637);
             }
             oprot.writeListEnd();
           }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartNames()) {
         {
           oprot.writeI32(struct.partNames.size());
-          for (String _iter622 : struct.partNames)
+          for (String _iter638 : struct.partNames)
           {
-            oprot.writeString(_iter622);
+            oprot.writeString(_iter638);
           }
         }
       }
@@ -934,13 +934,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partNames = new ArrayList<String>(_list623.size);
-          String _elem624;
-          for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partNames = new ArrayList<String>(_list639.size);
+          String _elem640;
+          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
           {
-            _elem624 = iprot.readString();
-            struct.partNames.add(_elem624);
+            _elem640 = iprot.readString();
+            struct.partNames.add(_elem640);
           }
         }
         struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index b87f65f..1d7677c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list952.size);
-                FieldSchema _elem953;
-                for (int _i954 = 0; _i954 < _list952.size; ++_i954)
+                org.apache.thrift.protocol.TList _list968 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list968.size);
+                FieldSchema _elem969;
+                for (int _i970 = 0; _i970 < _list968.size; ++_i970)
                 {
-                  _elem953 = new FieldSchema();
-                  _elem953.read(iprot);
-                  struct.cols.add(_elem953);
+                  _elem969 = new FieldSchema();
+                  _elem969.read(iprot);
+                  struct.cols.add(_elem969);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter955 : struct.cols)
+          for (FieldSchema _iter971 : struct.cols)
           {
-            _iter955.write(oprot);
+            _iter971.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter956 : struct.cols)
+          for (FieldSchema _iter972 : struct.cols)
           {
-            _iter956.write(oprot);
+            _iter972.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list957.size);
-          FieldSchema _elem958;
-          for (int _i959 = 0; _i959 < _list957.size; ++_i959)
+          org.apache.thrift.protocol.TList _list973 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list973.size);
+          FieldSchema _elem974;
+          for (int _i975 = 0; _i975 < _list973.size; ++_i975)
           {
-            _elem958 = new FieldSchema();
-            _elem958.read(iprot);
-            struct.cols.add(_elem958);
+            _elem974 = new FieldSchema();
+            _elem974.read(iprot);
+            struct.cols.add(_elem974);
           }
         }
         struct.setColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index 2a7b3eb..5fcc0dd 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list716.size);
-                ShowCompactResponseElement _elem717;
-                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list732.size);
+                ShowCompactResponseElement _elem733;
+                for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                 {
-                  _elem717 = new ShowCompactResponseElement();
-                  _elem717.read(iprot);
-                  struct.compacts.add(_elem717);
+                  _elem733 = new ShowCompactResponseElement();
+                  _elem733.read(iprot);
+                  struct.compacts.add(_elem733);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter719 : struct.compacts)
+          for (ShowCompactResponseElement _iter735 : struct.compacts)
           {
-            _iter719.write(oprot);
+            _iter735.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter720 : struct.compacts)
+        for (ShowCompactResponseElement _iter736 : struct.compacts)
         {
-          _iter720.write(oprot);
+          _iter736.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list721.size);
-        ShowCompactResponseElement _elem722;
-        for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+        org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list737.size);
+        ShowCompactResponseElement _elem738;
+        for (int _i739 = 0; _i739 < _list737.size; ++_i739)
         {
-          _elem722 = new ShowCompactResponseElement();
-          _elem722.read(iprot);
-          struct.compacts.add(_elem722);
+          _elem738 = new ShowCompactResponseElement();
+          _elem738.read(iprot);
+          struct.compacts.add(_elem738);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 477bf67..3db5327 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list682.size);
-                ShowLocksResponseElement _elem683;
-                for (int _i684 = 0; _i684 < _list682.size; ++_i684)
+                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list698.size);
+                ShowLocksResponseElement _elem699;
+                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
                 {
-                  _elem683 = new ShowLocksResponseElement();
-                  _elem683.read(iprot);
-                  struct.locks.add(_elem683);
+                  _elem699 = new ShowLocksResponseElement();
+                  _elem699.read(iprot);
+                  struct.locks.add(_elem699);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter685 : struct.locks)
+          for (ShowLocksResponseElement _iter701 : struct.locks)
           {
-            _iter685.write(oprot);
+            _iter701.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter686 : struct.locks)
+          for (ShowLocksResponseElement _iter702 : struct.locks)
           {
-            _iter686.write(oprot);
+            _iter702.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list687.size);
-          ShowLocksResponseElement _elem688;
-          for (int _i689 = 0; _i689 < _list687.size; ++_i689)
+          org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list703.size);
+          ShowLocksResponseElement _elem704;
+          for (int _i705 = 0; _i705 < _list703.size; ++_i705)
           {
-            _elem688 = new ShowLocksResponseElement();
-            _elem688.read(iprot);
-            struct.locks.add(_elem688);
+            _elem704 = new ShowLocksResponseElement();
+            _elem704.read(iprot);
+            struct.locks.add(_elem704);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
index 118f9db..e5ba13d 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
@@ -708,13 +708,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INVALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
-                struct.invalidWriteIds = new ArrayList<Long>(_list634.size);
-                long _elem635;
-                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.invalidWriteIds = new ArrayList<Long>(_list650.size);
+                long _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem635 = iprot.readI64();
-                  struct.invalidWriteIds.add(_elem635);
+                  _elem651 = iprot.readI64();
+                  struct.invalidWriteIds.add(_elem651);
                 }
                 iprot.readListEnd();
               }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INVALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalidWriteIds.size()));
-          for (long _iter637 : struct.invalidWriteIds)
+          for (long _iter653 : struct.invalidWriteIds)
           {
-            oprot.writeI64(_iter637);
+            oprot.writeI64(_iter653);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.writeIdHighWaterMark);
       {
         oprot.writeI32(struct.invalidWriteIds.size());
-        for (long _iter638 : struct.invalidWriteIds)
+        for (long _iter654 : struct.invalidWriteIds)
         {
-          oprot.writeI64(_iter638);
+          oprot.writeI64(_iter654);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -827,13 +827,13 @@ import org.slf4j.LoggerFactory;
       struct.writeIdHighWaterMark = iprot.readI64();
       struct.setWriteIdHighWaterMarkIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.invalidWriteIds = new ArrayList<Long>(_list639.size);
-        long _elem640;
-        for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.invalidWriteIds = new ArrayList<Long>(_list655.size);
+        long _elem656;
+        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
         {
-          _elem640 = iprot.readI64();
-          struct.invalidWriteIds.add(_elem640);
+          _elem656 = iprot.readI64();
+          struct.invalidWriteIds.add(_elem656);
         }
       }
       struct.setInvalidWriteIdsIsSet(true);


[05/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index b94dd25..333a2d9 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -711,6 +711,13 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function get_partitions_by_names($db_name, $tbl_name, array $names);
   /**
+   * @param \metastore\GetPartitionsByNamesRequest $req
+   * @return \metastore\GetPartitionsByNamesResult
+   * @throws \metastore\MetaException
+   * @throws \metastore\NoSuchObjectException
+   */
+  public function get_partitions_by_names_req(\metastore\GetPartitionsByNamesRequest $req);
+  /**
    * @param string $db_name
    * @param string $tbl_name
    * @param \metastore\Partition $new_part
@@ -6485,6 +6492,63 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_partitions_by_names failed: unknown result");
   }
 
+  public function get_partitions_by_names_req(\metastore\GetPartitionsByNamesRequest $req)
+  {
+    $this->send_get_partitions_by_names_req($req);
+    return $this->recv_get_partitions_by_names_req();
+  }
+
+  public function send_get_partitions_by_names_req(\metastore\GetPartitionsByNamesRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_partitions_by_names_req_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_partitions_by_names_req', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_partitions_by_names_req', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_partitions_by_names_req()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_partitions_by_names_req_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_partitions_by_names_req_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_partitions_by_names_req failed: unknown result");
+  }
+
   public function alter_partition($db_name, $tbl_name, \metastore\Partition $new_part)
   {
     $this->send_alter_partition($db_name, $tbl_name, $new_part);
@@ -16344,14 +16408,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size896 = 0;
-            $_etype899 = 0;
-            $xfer += $input->readListBegin($_etype899, $_size896);
-            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
+            $_size910 = 0;
+            $_etype913 = 0;
+            $xfer += $input->readListBegin($_etype913, $_size910);
+            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
             {
-              $elem901 = null;
-              $xfer += $input->readString($elem901);
-              $this->success []= $elem901;
+              $elem915 = null;
+              $xfer += $input->readString($elem915);
+              $this->success []= $elem915;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16387,9 +16451,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter902)
+          foreach ($this->success as $iter916)
           {
-            $xfer += $output->writeString($iter902);
+            $xfer += $output->writeString($iter916);
           }
         }
         $output->writeListEnd();
@@ -16520,14 +16584,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size903 = 0;
-            $_etype906 = 0;
-            $xfer += $input->readListBegin($_etype906, $_size903);
-            for ($_i907 = 0; $_i907 < $_size903; ++$_i907)
+            $_size917 = 0;
+            $_etype920 = 0;
+            $xfer += $input->readListBegin($_etype920, $_size917);
+            for ($_i921 = 0; $_i921 < $_size917; ++$_i921)
             {
-              $elem908 = null;
-              $xfer += $input->readString($elem908);
-              $this->success []= $elem908;
+              $elem922 = null;
+              $xfer += $input->readString($elem922);
+              $this->success []= $elem922;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16563,9 +16627,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter909)
+          foreach ($this->success as $iter923)
           {
-            $xfer += $output->writeString($iter909);
+            $xfer += $output->writeString($iter923);
           }
         }
         $output->writeListEnd();
@@ -17566,18 +17630,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size910 = 0;
-            $_ktype911 = 0;
-            $_vtype912 = 0;
-            $xfer += $input->readMapBegin($_ktype911, $_vtype912, $_size910);
-            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
+            $_size924 = 0;
+            $_ktype925 = 0;
+            $_vtype926 = 0;
+            $xfer += $input->readMapBegin($_ktype925, $_vtype926, $_size924);
+            for ($_i928 = 0; $_i928 < $_size924; ++$_i928)
             {
-              $key915 = '';
-              $val916 = new \metastore\Type();
-              $xfer += $input->readString($key915);
-              $val916 = new \metastore\Type();
-              $xfer += $val916->read($input);
-              $this->success[$key915] = $val916;
+              $key929 = '';
+              $val930 = new \metastore\Type();
+              $xfer += $input->readString($key929);
+              $val930 = new \metastore\Type();
+              $xfer += $val930->read($input);
+              $this->success[$key929] = $val930;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -17613,10 +17677,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter917 => $viter918)
+          foreach ($this->success as $kiter931 => $viter932)
           {
-            $xfer += $output->writeString($kiter917);
-            $xfer += $viter918->write($output);
+            $xfer += $output->writeString($kiter931);
+            $xfer += $viter932->write($output);
           }
         }
         $output->writeMapEnd();
@@ -17820,15 +17884,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size919 = 0;
-            $_etype922 = 0;
-            $xfer += $input->readListBegin($_etype922, $_size919);
-            for ($_i923 = 0; $_i923 < $_size919; ++$_i923)
+            $_size933 = 0;
+            $_etype936 = 0;
+            $xfer += $input->readListBegin($_etype936, $_size933);
+            for ($_i937 = 0; $_i937 < $_size933; ++$_i937)
             {
-              $elem924 = null;
-              $elem924 = new \metastore\FieldSchema();
-              $xfer += $elem924->read($input);
-              $this->success []= $elem924;
+              $elem938 = null;
+              $elem938 = new \metastore\FieldSchema();
+              $xfer += $elem938->read($input);
+              $this->success []= $elem938;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17880,9 +17944,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter925)
+          foreach ($this->success as $iter939)
           {
-            $xfer += $iter925->write($output);
+            $xfer += $iter939->write($output);
           }
         }
         $output->writeListEnd();
@@ -18124,15 +18188,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size926 = 0;
-            $_etype929 = 0;
-            $xfer += $input->readListBegin($_etype929, $_size926);
-            for ($_i930 = 0; $_i930 < $_size926; ++$_i930)
+            $_size940 = 0;
+            $_etype943 = 0;
+            $xfer += $input->readListBegin($_etype943, $_size940);
+            for ($_i944 = 0; $_i944 < $_size940; ++$_i944)
             {
-              $elem931 = null;
-              $elem931 = new \metastore\FieldSchema();
-              $xfer += $elem931->read($input);
-              $this->success []= $elem931;
+              $elem945 = null;
+              $elem945 = new \metastore\FieldSchema();
+              $xfer += $elem945->read($input);
+              $this->success []= $elem945;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18184,9 +18248,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter932)
+          foreach ($this->success as $iter946)
           {
-            $xfer += $iter932->write($output);
+            $xfer += $iter946->write($output);
           }
         }
         $output->writeListEnd();
@@ -18400,15 +18464,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size933 = 0;
-            $_etype936 = 0;
-            $xfer += $input->readListBegin($_etype936, $_size933);
-            for ($_i937 = 0; $_i937 < $_size933; ++$_i937)
+            $_size947 = 0;
+            $_etype950 = 0;
+            $xfer += $input->readListBegin($_etype950, $_size947);
+            for ($_i951 = 0; $_i951 < $_size947; ++$_i951)
             {
-              $elem938 = null;
-              $elem938 = new \metastore\FieldSchema();
-              $xfer += $elem938->read($input);
-              $this->success []= $elem938;
+              $elem952 = null;
+              $elem952 = new \metastore\FieldSchema();
+              $xfer += $elem952->read($input);
+              $this->success []= $elem952;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18460,9 +18524,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter939)
+          foreach ($this->success as $iter953)
           {
-            $xfer += $iter939->write($output);
+            $xfer += $iter953->write($output);
           }
         }
         $output->writeListEnd();
@@ -18704,15 +18768,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size940 = 0;
-            $_etype943 = 0;
-            $xfer += $input->readListBegin($_etype943, $_size940);
-            for ($_i944 = 0; $_i944 < $_size940; ++$_i944)
+            $_size954 = 0;
+            $_etype957 = 0;
+            $xfer += $input->readListBegin($_etype957, $_size954);
+            for ($_i958 = 0; $_i958 < $_size954; ++$_i958)
             {
-              $elem945 = null;
-              $elem945 = new \metastore\FieldSchema();
-              $xfer += $elem945->read($input);
-              $this->success []= $elem945;
+              $elem959 = null;
+              $elem959 = new \metastore\FieldSchema();
+              $xfer += $elem959->read($input);
+              $this->success []= $elem959;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18764,9 +18828,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter946)
+          foreach ($this->success as $iter960)
           {
-            $xfer += $iter946->write($output);
+            $xfer += $iter960->write($output);
           }
         }
         $output->writeListEnd();
@@ -19438,15 +19502,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size947 = 0;
-            $_etype950 = 0;
-            $xfer += $input->readListBegin($_etype950, $_size947);
-            for ($_i951 = 0; $_i951 < $_size947; ++$_i951)
+            $_size961 = 0;
+            $_etype964 = 0;
+            $xfer += $input->readListBegin($_etype964, $_size961);
+            for ($_i965 = 0; $_i965 < $_size961; ++$_i965)
             {
-              $elem952 = null;
-              $elem952 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem952->read($input);
-              $this->primaryKeys []= $elem952;
+              $elem966 = null;
+              $elem966 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem966->read($input);
+              $this->primaryKeys []= $elem966;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19456,15 +19520,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size953 = 0;
-            $_etype956 = 0;
-            $xfer += $input->readListBegin($_etype956, $_size953);
-            for ($_i957 = 0; $_i957 < $_size953; ++$_i957)
+            $_size967 = 0;
+            $_etype970 = 0;
+            $xfer += $input->readListBegin($_etype970, $_size967);
+            for ($_i971 = 0; $_i971 < $_size967; ++$_i971)
             {
-              $elem958 = null;
-              $elem958 = new \metastore\SQLForeignKey();
-              $xfer += $elem958->read($input);
-              $this->foreignKeys []= $elem958;
+              $elem972 = null;
+              $elem972 = new \metastore\SQLForeignKey();
+              $xfer += $elem972->read($input);
+              $this->foreignKeys []= $elem972;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19474,15 +19538,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size959 = 0;
-            $_etype962 = 0;
-            $xfer += $input->readListBegin($_etype962, $_size959);
-            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem964 = null;
-              $elem964 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem964->read($input);
-              $this->uniqueConstraints []= $elem964;
+              $elem978 = null;
+              $elem978 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem978->read($input);
+              $this->uniqueConstraints []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19492,15 +19556,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size965 = 0;
-            $_etype968 = 0;
-            $xfer += $input->readListBegin($_etype968, $_size965);
-            for ($_i969 = 0; $_i969 < $_size965; ++$_i969)
+            $_size979 = 0;
+            $_etype982 = 0;
+            $xfer += $input->readListBegin($_etype982, $_size979);
+            for ($_i983 = 0; $_i983 < $_size979; ++$_i983)
             {
-              $elem970 = null;
-              $elem970 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem970->read($input);
-              $this->notNullConstraints []= $elem970;
+              $elem984 = null;
+              $elem984 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem984->read($input);
+              $this->notNullConstraints []= $elem984;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19510,15 +19574,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size971 = 0;
-            $_etype974 = 0;
-            $xfer += $input->readListBegin($_etype974, $_size971);
-            for ($_i975 = 0; $_i975 < $_size971; ++$_i975)
+            $_size985 = 0;
+            $_etype988 = 0;
+            $xfer += $input->readListBegin($_etype988, $_size985);
+            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
             {
-              $elem976 = null;
-              $elem976 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem976->read($input);
-              $this->defaultConstraints []= $elem976;
+              $elem990 = null;
+              $elem990 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem990->read($input);
+              $this->defaultConstraints []= $elem990;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19528,15 +19592,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size977 = 0;
-            $_etype980 = 0;
-            $xfer += $input->readListBegin($_etype980, $_size977);
-            for ($_i981 = 0; $_i981 < $_size977; ++$_i981)
+            $_size991 = 0;
+            $_etype994 = 0;
+            $xfer += $input->readListBegin($_etype994, $_size991);
+            for ($_i995 = 0; $_i995 < $_size991; ++$_i995)
             {
-              $elem982 = null;
-              $elem982 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem982->read($input);
-              $this->checkConstraints []= $elem982;
+              $elem996 = null;
+              $elem996 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem996->read($input);
+              $this->checkConstraints []= $elem996;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19572,9 +19636,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter983)
+          foreach ($this->primaryKeys as $iter997)
           {
-            $xfer += $iter983->write($output);
+            $xfer += $iter997->write($output);
           }
         }
         $output->writeListEnd();
@@ -19589,9 +19653,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter984)
+          foreach ($this->foreignKeys as $iter998)
           {
-            $xfer += $iter984->write($output);
+            $xfer += $iter998->write($output);
           }
         }
         $output->writeListEnd();
@@ -19606,9 +19670,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter985)
+          foreach ($this->uniqueConstraints as $iter999)
           {
-            $xfer += $iter985->write($output);
+            $xfer += $iter999->write($output);
           }
         }
         $output->writeListEnd();
@@ -19623,9 +19687,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter986)
+          foreach ($this->notNullConstraints as $iter1000)
           {
-            $xfer += $iter986->write($output);
+            $xfer += $iter1000->write($output);
           }
         }
         $output->writeListEnd();
@@ -19640,9 +19704,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter987)
+          foreach ($this->defaultConstraints as $iter1001)
           {
-            $xfer += $iter987->write($output);
+            $xfer += $iter1001->write($output);
           }
         }
         $output->writeListEnd();
@@ -19657,9 +19721,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter988)
+          foreach ($this->checkConstraints as $iter1002)
           {
-            $xfer += $iter988->write($output);
+            $xfer += $iter1002->write($output);
           }
         }
         $output->writeListEnd();
@@ -21659,14 +21723,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size989 = 0;
-            $_etype992 = 0;
-            $xfer += $input->readListBegin($_etype992, $_size989);
-            for ($_i993 = 0; $_i993 < $_size989; ++$_i993)
+            $_size1003 = 0;
+            $_etype1006 = 0;
+            $xfer += $input->readListBegin($_etype1006, $_size1003);
+            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
             {
-              $elem994 = null;
-              $xfer += $input->readString($elem994);
-              $this->partNames []= $elem994;
+              $elem1008 = null;
+              $xfer += $input->readString($elem1008);
+              $this->partNames []= $elem1008;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21704,9 +21768,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter995)
+          foreach ($this->partNames as $iter1009)
           {
-            $xfer += $output->writeString($iter995);
+            $xfer += $output->writeString($iter1009);
           }
         }
         $output->writeListEnd();
@@ -22142,14 +22206,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size996 = 0;
-            $_etype999 = 0;
-            $xfer += $input->readListBegin($_etype999, $_size996);
-            for ($_i1000 = 0; $_i1000 < $_size996; ++$_i1000)
+            $_size1010 = 0;
+            $_etype1013 = 0;
+            $xfer += $input->readListBegin($_etype1013, $_size1010);
+            for ($_i1014 = 0; $_i1014 < $_size1010; ++$_i1014)
             {
-              $elem1001 = null;
-              $xfer += $input->readString($elem1001);
-              $this->success []= $elem1001;
+              $elem1015 = null;
+              $xfer += $input->readString($elem1015);
+              $this->success []= $elem1015;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22185,9 +22249,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1002)
+          foreach ($this->success as $iter1016)
           {
-            $xfer += $output->writeString($iter1002);
+            $xfer += $output->writeString($iter1016);
           }
         }
         $output->writeListEnd();
@@ -22389,14 +22453,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1003 = 0;
-            $_etype1006 = 0;
-            $xfer += $input->readListBegin($_etype1006, $_size1003);
-            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
+            $_size1017 = 0;
+            $_etype1020 = 0;
+            $xfer += $input->readListBegin($_etype1020, $_size1017);
+            for ($_i1021 = 0; $_i1021 < $_size1017; ++$_i1021)
             {
-              $elem1008 = null;
-              $xfer += $input->readString($elem1008);
-              $this->success []= $elem1008;
+              $elem1022 = null;
+              $xfer += $input->readString($elem1022);
+              $this->success []= $elem1022;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22432,9 +22496,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1009)
+          foreach ($this->success as $iter1023)
           {
-            $xfer += $output->writeString($iter1009);
+            $xfer += $output->writeString($iter1023);
           }
         }
         $output->writeListEnd();
@@ -22590,14 +22654,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1010 = 0;
-            $_etype1013 = 0;
-            $xfer += $input->readListBegin($_etype1013, $_size1010);
-            for ($_i1014 = 0; $_i1014 < $_size1010; ++$_i1014)
+            $_size1024 = 0;
+            $_etype1027 = 0;
+            $xfer += $input->readListBegin($_etype1027, $_size1024);
+            for ($_i1028 = 0; $_i1028 < $_size1024; ++$_i1028)
             {
-              $elem1015 = null;
-              $xfer += $input->readString($elem1015);
-              $this->success []= $elem1015;
+              $elem1029 = null;
+              $xfer += $input->readString($elem1029);
+              $this->success []= $elem1029;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22633,9 +22697,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1016)
+          foreach ($this->success as $iter1030)
           {
-            $xfer += $output->writeString($iter1016);
+            $xfer += $output->writeString($iter1030);
           }
         }
         $output->writeListEnd();
@@ -22740,14 +22804,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size1017 = 0;
-            $_etype1020 = 0;
-            $xfer += $input->readListBegin($_etype1020, $_size1017);
-            for ($_i1021 = 0; $_i1021 < $_size1017; ++$_i1021)
+            $_size1031 = 0;
+            $_etype1034 = 0;
+            $xfer += $input->readListBegin($_etype1034, $_size1031);
+            for ($_i1035 = 0; $_i1035 < $_size1031; ++$_i1035)
             {
-              $elem1022 = null;
-              $xfer += $input->readString($elem1022);
-              $this->tbl_types []= $elem1022;
+              $elem1036 = null;
+              $xfer += $input->readString($elem1036);
+              $this->tbl_types []= $elem1036;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22785,9 +22849,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter1023)
+          foreach ($this->tbl_types as $iter1037)
           {
-            $xfer += $output->writeString($iter1023);
+            $xfer += $output->writeString($iter1037);
           }
         }
         $output->writeListEnd();
@@ -22864,15 +22928,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1024 = 0;
-            $_etype1027 = 0;
-            $xfer += $input->readListBegin($_etype1027, $_size1024);
-            for ($_i1028 = 0; $_i1028 < $_size1024; ++$_i1028)
+            $_size1038 = 0;
+            $_etype1041 = 0;
+            $xfer += $input->readListBegin($_etype1041, $_size1038);
+            for ($_i1042 = 0; $_i1042 < $_size1038; ++$_i1042)
             {
-              $elem1029 = null;
-              $elem1029 = new \metastore\TableMeta();
-              $xfer += $elem1029->read($input);
-              $this->success []= $elem1029;
+              $elem1043 = null;
+              $elem1043 = new \metastore\TableMeta();
+              $xfer += $elem1043->read($input);
+              $this->success []= $elem1043;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22908,9 +22972,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1030)
+          foreach ($this->success as $iter1044)
           {
-            $xfer += $iter1030->write($output);
+            $xfer += $iter1044->write($output);
           }
         }
         $output->writeListEnd();
@@ -23066,14 +23130,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1031 = 0;
-            $_etype1034 = 0;
-            $xfer += $input->readListBegin($_etype1034, $_size1031);
-            for ($_i1035 = 0; $_i1035 < $_size1031; ++$_i1035)
+            $_size1045 = 0;
+            $_etype1048 = 0;
+            $xfer += $input->readListBegin($_etype1048, $_size1045);
+            for ($_i1049 = 0; $_i1049 < $_size1045; ++$_i1049)
             {
-              $elem1036 = null;
-              $xfer += $input->readString($elem1036);
-              $this->success []= $elem1036;
+              $elem1050 = null;
+              $xfer += $input->readString($elem1050);
+              $this->success []= $elem1050;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23109,9 +23173,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1037)
+          foreach ($this->success as $iter1051)
           {
-            $xfer += $output->writeString($iter1037);
+            $xfer += $output->writeString($iter1051);
           }
         }
         $output->writeListEnd();
@@ -23426,14 +23490,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size1038 = 0;
-            $_etype1041 = 0;
-            $xfer += $input->readListBegin($_etype1041, $_size1038);
-            for ($_i1042 = 0; $_i1042 < $_size1038; ++$_i1042)
+            $_size1052 = 0;
+            $_etype1055 = 0;
+            $xfer += $input->readListBegin($_etype1055, $_size1052);
+            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
             {
-              $elem1043 = null;
-              $xfer += $input->readString($elem1043);
-              $this->tbl_names []= $elem1043;
+              $elem1057 = null;
+              $xfer += $input->readString($elem1057);
+              $this->tbl_names []= $elem1057;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23466,9 +23530,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter1044)
+          foreach ($this->tbl_names as $iter1058)
           {
-            $xfer += $output->writeString($iter1044);
+            $xfer += $output->writeString($iter1058);
           }
         }
         $output->writeListEnd();
@@ -23533,15 +23597,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1045 = 0;
-            $_etype1048 = 0;
-            $xfer += $input->readListBegin($_etype1048, $_size1045);
-            for ($_i1049 = 0; $_i1049 < $_size1045; ++$_i1049)
+            $_size1059 = 0;
+            $_etype1062 = 0;
+            $xfer += $input->readListBegin($_etype1062, $_size1059);
+            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
             {
-              $elem1050 = null;
-              $elem1050 = new \metastore\Table();
-              $xfer += $elem1050->read($input);
-              $this->success []= $elem1050;
+              $elem1064 = null;
+              $elem1064 = new \metastore\Table();
+              $xfer += $elem1064->read($input);
+              $this->success []= $elem1064;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23569,9 +23633,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1051)
+          foreach ($this->success as $iter1065)
           {
-            $xfer += $iter1051->write($output);
+            $xfer += $iter1065->write($output);
           }
         }
         $output->writeListEnd();
@@ -24771,14 +24835,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1052 = 0;
-            $_etype1055 = 0;
-            $xfer += $input->readListBegin($_etype1055, $_size1052);
-            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
+            $_size1066 = 0;
+            $_etype1069 = 0;
+            $xfer += $input->readListBegin($_etype1069, $_size1066);
+            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
             {
-              $elem1057 = null;
-              $xfer += $input->readString($elem1057);
-              $this->success []= $elem1057;
+              $elem1071 = null;
+              $xfer += $input->readString($elem1071);
+              $this->success []= $elem1071;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24830,9 +24894,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1058)
+          foreach ($this->success as $iter1072)
           {
-            $xfer += $output->writeString($iter1058);
+            $xfer += $output->writeString($iter1072);
           }
         }
         $output->writeListEnd();
@@ -26355,15 +26419,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1059 = 0;
-            $_etype1062 = 0;
-            $xfer += $input->readListBegin($_etype1062, $_size1059);
-            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
+            $_size1073 = 0;
+            $_etype1076 = 0;
+            $xfer += $input->readListBegin($_etype1076, $_size1073);
+            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
             {
-              $elem1064 = null;
-              $elem1064 = new \metastore\Partition();
-              $xfer += $elem1064->read($input);
-              $this->new_parts []= $elem1064;
+              $elem1078 = null;
+              $elem1078 = new \metastore\Partition();
+              $xfer += $elem1078->read($input);
+              $this->new_parts []= $elem1078;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26391,9 +26455,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1065)
+          foreach ($this->new_parts as $iter1079)
           {
-            $xfer += $iter1065->write($output);
+            $xfer += $iter1079->write($output);
           }
         }
         $output->writeListEnd();
@@ -26608,15 +26672,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1066 = 0;
-            $_etype1069 = 0;
-            $xfer += $input->readListBegin($_etype1069, $_size1066);
-            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
+            $_size1080 = 0;
+            $_etype1083 = 0;
+            $xfer += $input->readListBegin($_etype1083, $_size1080);
+            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
             {
-              $elem1071 = null;
-              $elem1071 = new \metastore\PartitionSpec();
-              $xfer += $elem1071->read($input);
-              $this->new_parts []= $elem1071;
+              $elem1085 = null;
+              $elem1085 = new \metastore\PartitionSpec();
+              $xfer += $elem1085->read($input);
+              $this->new_parts []= $elem1085;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26644,9 +26708,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1072)
+          foreach ($this->new_parts as $iter1086)
           {
-            $xfer += $iter1072->write($output);
+            $xfer += $iter1086->write($output);
           }
         }
         $output->writeListEnd();
@@ -26896,14 +26960,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1073 = 0;
-            $_etype1076 = 0;
-            $xfer += $input->readListBegin($_etype1076, $_size1073);
-            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1078 = null;
-              $xfer += $input->readString($elem1078);
-              $this->part_vals []= $elem1078;
+              $elem1092 = null;
+              $xfer += $input->readString($elem1092);
+              $this->part_vals []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26941,9 +27005,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1079)
+          foreach ($this->part_vals as $iter1093)
           {
-            $xfer += $output->writeString($iter1079);
+            $xfer += $output->writeString($iter1093);
           }
         }
         $output->writeListEnd();
@@ -27445,14 +27509,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1080 = 0;
-            $_etype1083 = 0;
-            $xfer += $input->readListBegin($_etype1083, $_size1080);
-            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
+            $_size1094 = 0;
+            $_etype1097 = 0;
+            $xfer += $input->readListBegin($_etype1097, $_size1094);
+            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
             {
-              $elem1085 = null;
-              $xfer += $input->readString($elem1085);
-              $this->part_vals []= $elem1085;
+              $elem1099 = null;
+              $xfer += $input->readString($elem1099);
+              $this->part_vals []= $elem1099;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27498,9 +27562,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1086)
+          foreach ($this->part_vals as $iter1100)
           {
-            $xfer += $output->writeString($iter1086);
+            $xfer += $output->writeString($iter1100);
           }
         }
         $output->writeListEnd();
@@ -28354,14 +28418,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1087 = 0;
-            $_etype1090 = 0;
-            $xfer += $input->readListBegin($_etype1090, $_size1087);
-            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
+            $_size1101 = 0;
+            $_etype1104 = 0;
+            $xfer += $input->readListBegin($_etype1104, $_size1101);
+            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
             {
-              $elem1092 = null;
-              $xfer += $input->readString($elem1092);
-              $this->part_vals []= $elem1092;
+              $elem1106 = null;
+              $xfer += $input->readString($elem1106);
+              $this->part_vals []= $elem1106;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28406,9 +28470,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1093)
+          foreach ($this->part_vals as $iter1107)
           {
-            $xfer += $output->writeString($iter1093);
+            $xfer += $output->writeString($iter1107);
           }
         }
         $output->writeListEnd();
@@ -28661,14 +28725,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1094 = 0;
-            $_etype1097 = 0;
-            $xfer += $input->readListBegin($_etype1097, $_size1094);
-            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
+            $_size1108 = 0;
+            $_etype1111 = 0;
+            $xfer += $input->readListBegin($_etype1111, $_size1108);
+            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
             {
-              $elem1099 = null;
-              $xfer += $input->readString($elem1099);
-              $this->part_vals []= $elem1099;
+              $elem1113 = null;
+              $xfer += $input->readString($elem1113);
+              $this->part_vals []= $elem1113;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28721,9 +28785,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1100)
+          foreach ($this->part_vals as $iter1114)
           {
-            $xfer += $output->writeString($iter1100);
+            $xfer += $output->writeString($iter1114);
           }
         }
         $output->writeListEnd();
@@ -29737,14 +29801,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1101 = 0;
-            $_etype1104 = 0;
-            $xfer += $input->readListBegin($_etype1104, $_size1101);
-            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
+            $_size1115 = 0;
+            $_etype1118 = 0;
+            $xfer += $input->readListBegin($_etype1118, $_size1115);
+            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
             {
-              $elem1106 = null;
-              $xfer += $input->readString($elem1106);
-              $this->part_vals []= $elem1106;
+              $elem1120 = null;
+              $xfer += $input->readString($elem1120);
+              $this->part_vals []= $elem1120;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29782,9 +29846,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1107)
+          foreach ($this->part_vals as $iter1121)
           {
-            $xfer += $output->writeString($iter1107);
+            $xfer += $output->writeString($iter1121);
           }
         }
         $output->writeListEnd();
@@ -30026,17 +30090,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1108 = 0;
-            $_ktype1109 = 0;
-            $_vtype1110 = 0;
-            $xfer += $input->readMapBegin($_ktype1109, $_vtype1110, $_size1108);
-            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
+            $_size1122 = 0;
+            $_ktype1123 = 0;
+            $_vtype1124 = 0;
+            $xfer += $input->readMapBegin($_ktype1123, $_vtype1124, $_size1122);
+            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
             {
-              $key1113 = '';
-              $val1114 = '';
-              $xfer += $input->readString($key1113);
-              $xfer += $input->readString($val1114);
-              $this->partitionSpecs[$key1113] = $val1114;
+              $key1127 = '';
+              $val1128 = '';
+              $xfer += $input->readString($key1127);
+              $xfer += $input->readString($val1128);
+              $this->partitionSpecs[$key1127] = $val1128;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -30092,10 +30156,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1115 => $viter1116)
+          foreach ($this->partitionSpecs as $kiter1129 => $viter1130)
           {
-            $xfer += $output->writeString($kiter1115);
-            $xfer += $output->writeString($viter1116);
+            $xfer += $output->writeString($kiter1129);
+            $xfer += $output->writeString($viter1130);
           }
         }
         $output->writeMapEnd();
@@ -30407,17 +30471,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1117 = 0;
-            $_ktype1118 = 0;
-            $_vtype1119 = 0;
-            $xfer += $input->readMapBegin($_ktype1118, $_vtype1119, $_size1117);
-            for ($_i1121 = 0; $_i1121 < $_size1117; ++$_i1121)
+            $_size1131 = 0;
+            $_ktype1132 = 0;
+            $_vtype1133 = 0;
+            $xfer += $input->readMapBegin($_ktype1132, $_vtype1133, $_size1131);
+            for ($_i1135 = 0; $_i1135 < $_size1131; ++$_i1135)
             {
-              $key1122 = '';
-              $val1123 = '';
-              $xfer += $input->readString($key1122);
-              $xfer += $input->readString($val1123);
-              $this->partitionSpecs[$key1122] = $val1123;
+              $key1136 = '';
+              $val1137 = '';
+              $xfer += $input->readString($key1136);
+              $xfer += $input->readString($val1137);
+              $this->partitionSpecs[$key1136] = $val1137;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -30473,10 +30537,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1124 => $viter1125)
+          foreach ($this->partitionSpecs as $kiter1138 => $viter1139)
           {
-            $xfer += $output->writeString($kiter1124);
-            $xfer += $output->writeString($viter1125);
+            $xfer += $output->writeString($kiter1138);
+            $xfer += $output->writeString($viter1139);
           }
         }
         $output->writeMapEnd();
@@ -30609,15 +30673,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1126 = 0;
-            $_etype1129 = 0;
-            $xfer += $input->readListBegin($_etype1129, $_size1126);
-            for ($_i1130 = 0; $_i1130 < $_size1126; ++$_i1130)
+            $_size1140 = 0;
+            $_etype1143 = 0;
+            $xfer += $input->readListBegin($_etype1143, $_size1140);
+            for ($_i1144 = 0; $_i1144 < $_size1140; ++$_i1144)
             {
-              $elem1131 = null;
-              $elem1131 = new \metastore\Partition();
-              $xfer += $elem1131->read($input);
-              $this->success []= $elem1131;
+              $elem1145 = null;
+              $elem1145 = new \metastore\Partition();
+              $xfer += $elem1145->read($input);
+              $this->success []= $elem1145;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30677,9 +30741,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1132)
+          foreach ($this->success as $iter1146)
           {
-            $xfer += $iter1132->write($output);
+            $xfer += $iter1146->write($output);
           }
         }
         $output->writeListEnd();
@@ -30825,14 +30889,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1133 = 0;
-            $_etype1136 = 0;
-            $xfer += $input->readListBegin($_etype1136, $_size1133);
-            for ($_i1137 = 0; $_i1137 < $_size1133; ++$_i1137)
+            $_size1147 = 0;
+            $_etype1150 = 0;
+            $xfer += $input->readListBegin($_etype1150, $_size1147);
+            for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151)
             {
-              $elem1138 = null;
-              $xfer += $input->readString($elem1138);
-              $this->part_vals []= $elem1138;
+              $elem1152 = null;
+              $xfer += $input->readString($elem1152);
+              $this->part_vals []= $elem1152;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30849,14 +30913,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1139 = 0;
-            $_etype1142 = 0;
-            $xfer += $input->readListBegin($_etype1142, $_size1139);
-            for ($_i1143 = 0; $_i1143 < $_size1139; ++$_i1143)
+            $_size1153 = 0;
+            $_etype1156 = 0;
+            $xfer += $input->readListBegin($_etype1156, $_size1153);
+            for ($_i1157 = 0; $_i1157 < $_size1153; ++$_i1157)
             {
-              $elem1144 = null;
-              $xfer += $input->readString($elem1144);
-              $this->group_names []= $elem1144;
+              $elem1158 = null;
+              $xfer += $input->readString($elem1158);
+              $this->group_names []= $elem1158;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30894,9 +30958,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1145)
+          foreach ($this->part_vals as $iter1159)
           {
-            $xfer += $output->writeString($iter1145);
+            $xfer += $output->writeString($iter1159);
           }
         }
         $output->writeListEnd();
@@ -30916,9 +30980,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1146)
+          foreach ($this->group_names as $iter1160)
           {
-            $xfer += $output->writeString($iter1146);
+            $xfer += $output->writeString($iter1160);
           }
         }
         $output->writeListEnd();
@@ -31509,15 +31573,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1147 = 0;
-            $_etype1150 = 0;
-            $xfer += $input->readListBegin($_etype1150, $_size1147);
-            for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151)
+            $_size1161 = 0;
+            $_etype1164 = 0;
+            $xfer += $input->readListBegin($_etype1164, $_size1161);
+            for ($_i1165 = 0; $_i1165 < $_size1161; ++$_i1165)
             {
-              $elem1152 = null;
-              $elem1152 = new \metastore\Partition();
-              $xfer += $elem1152->read($input);
-              $this->success []= $elem1152;
+              $elem1166 = null;
+              $elem1166 = new \metastore\Partition();
+              $xfer += $elem1166->read($input);
+              $this->success []= $elem1166;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31561,9 +31625,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1153)
+          foreach ($this->success as $iter1167)
           {
-            $xfer += $iter1153->write($output);
+            $xfer += $iter1167->write($output);
           }
         }
         $output->writeListEnd();
@@ -31709,14 +31773,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1154 = 0;
-            $_etype1157 = 0;
-            $xfer += $input->readListBegin($_etype1157, $_size1154);
-            for ($_i1158 = 0; $_i1158 < $_size1154; ++$_i1158)
+            $_size1168 = 0;
+            $_etype1171 = 0;
+            $xfer += $input->readListBegin($_etype1171, $_size1168);
+            for ($_i1172 = 0; $_i1172 < $_size1168; ++$_i1172)
             {
-              $elem1159 = null;
-              $xfer += $input->readString($elem1159);
-              $this->group_names []= $elem1159;
+              $elem1173 = null;
+              $xfer += $input->readString($elem1173);
+              $this->group_names []= $elem1173;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31764,9 +31828,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1160)
+          foreach ($this->group_names as $iter1174)
           {
-            $xfer += $output->writeString($iter1160);
+            $xfer += $output->writeString($iter1174);
           }
         }
         $output->writeListEnd();
@@ -31855,15 +31919,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1161 = 0;
-            $_etype1164 = 0;
-            $xfer += $input->readListBegin($_etype1164, $_size1161);
-            for ($_i1165 = 0; $_i1165 < $_size1161; ++$_i1165)
+            $_size1175 = 0;
+            $_etype1178 = 0;
+            $xfer += $input->readListBegin($_etype1178, $_size1175);
+            for ($_i1179 = 0; $_i1179 < $_size1175; ++$_i1179)
             {
-              $elem1166 = null;
-              $elem1166 = new \metastore\Partition();
-              $xfer += $elem1166->read($input);
-              $this->success []= $elem1166;
+              $elem1180 = null;
+              $elem1180 = new \metastore\Partition();
+              $xfer += $elem1180->read($input);
+              $this->success []= $elem1180;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31907,9 +31971,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1167)
+          foreach ($this->success as $iter1181)
           {
-            $xfer += $iter1167->write($output);
+            $xfer += $iter1181->write($output);
           }
         }
         $output->writeListEnd();
@@ -32129,15 +32193,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1168 = 0;
-            $_etype1171 = 0;
-            $xfer += $input->readListBegin($_etype1171, $_size1168);
-            for ($_i1172 = 0; $_i1172 < $_size1168; ++$_i1172)
+            $_size1182 = 0;
+            $_etype1185 = 0;
+            $xfer += $input->readListBegin($_etype1185, $_size1182);
+            for ($_i1186 = 0; $_i1186 < $_size1182; ++$_i1186)
             {
-              $elem1173 = null;
-              $elem1173 = new \metastore\PartitionSpec();
-              $xfer += $elem1173->read($input);
-              $this->success []= $elem1173;
+              $elem1187 = null;
+              $elem1187 = new \metastore\PartitionSpec();
+              $xfer += $elem1187->read($input);
+              $this->success []= $elem1187;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32181,9 +32245,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1174)
+          foreach ($this->success as $iter1188)
           {
-            $xfer += $iter1174->write($output);
+            $xfer += $iter1188->write($output);
           }
         }
         $output->writeListEnd();
@@ -32402,14 +32466,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1175 = 0;
-            $_etype1178 = 0;
-            $xfer += $input->readListBegin($_etype1178, $_size1175);
-            for ($_i1179 = 0; $_i1179 < $_size1175; ++$_i1179)
+            $_size1189 = 0;
+            $_etype1192 = 0;
+            $xfer += $input->readListBegin($_etype1192, $_size1189);
+            for ($_i1193 = 0; $_i1193 < $_size1189; ++$_i1193)
             {
-              $elem1180 = null;
-              $xfer += $input->readString($elem1180);
-              $this->success []= $elem1180;
+              $elem1194 = null;
+              $xfer += $input->readString($elem1194);
+              $this->success []= $elem1194;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32453,9 +32517,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1181)
+          foreach ($this->success as $iter1195)
           {
-            $xfer += $output->writeString($iter1181);
+            $xfer += $output->writeString($iter1195);
           }
         }
         $output->writeListEnd();
@@ -32786,14 +32850,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1182 = 0;
-            $_etype1185 = 0;
-            $xfer += $input->readListBegin($_etype1185, $_size1182);
-            for ($_i1186 = 0; $_i1186 < $_size1182; ++$_i1186)
+            $_size1196 = 0;
+            $_etype1199 = 0;
+            $xfer += $input->readListBegin($_etype1199, $_size1196);
+            for ($_i1200 = 0; $_i1200 < $_size1196; ++$_i1200)
             {
-              $elem1187 = null;
-              $xfer += $input->readString($elem1187);
-              $this->part_vals []= $elem1187;
+              $elem1201 = null;
+              $xfer += $input->readString($elem1201);
+              $this->part_vals []= $elem1201;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32838,9 +32902,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1188)
+          foreach ($this->part_vals as $iter1202)
           {
-            $xfer += $output->writeString($iter1188);
+            $xfer += $output->writeString($iter1202);
           }
         }
         $output->writeListEnd();
@@ -32934,15 +32998,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1189 = 0;
-            $_etype1192 = 0;
-            $xfer += $input->readListBegin($_etype1192, $_size1189);
-            for ($_i1193 = 0; $_i1193 < $_size1189; ++$_i1193)
+            $_size1203 = 0;
+            $_etype1206 = 0;
+            $xfer += $input->readListBegin($_etype1206, $_size1203);
+            for ($_i1207 = 0; $_i1207 < $_size1203; ++$_i1207)
             {
-              $elem1194 = null;
-              $elem1194 = new \metastore\Partition();
-              $xfer += $elem1194->read($input);
-              $this->success []= $elem1194;
+              $elem1208 = null;
+              $elem1208 = new \metastore\Partition();
+              $xfer += $elem1208->read($input);
+              $this->success []= $elem1208;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32986,9 +33050,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1195)
+          foreach ($this->success as $iter1209)
           {
-            $xfer += $iter1195->write($output);
+            $xfer += $iter1209->write($output);
           }
         }
         $output->writeListEnd();
@@ -33135,14 +33199,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1196 = 0;
-            $_etype1199 = 0;
-            $xfer += $input->readListBegin($_etype1199, $_size1196);
-            for ($_i1200 = 0; $_i1200 < $_size1196; ++$_i1200)
+            $_size1210 = 0;
+            $_etype1213 = 0;
+            $xfer += $input->readListBegin($_etype1213, $_size1210);
+            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
             {
-              $elem1201 = null;
-              $xfer += $input->readString($elem1201);
-              $this->part_vals []= $elem1201;
+              $elem1215 = null;
+              $xfer += $input->readString($elem1215);
+              $this->part_vals []= $elem1215;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33166,14 +33230,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1202 = 0;
-            $_etype1205 = 0;
-            $xfer += $input->readListBegin($_etype1205, $_size1202);
-            for ($_i1206 = 0; $_i1206 < $_size1202; ++$_i1206)
+            $_size1216 = 0;
+            $_etype1219 = 0;
+            $xfer += $input->readListBegin($_etype1219, $_size1216);
+            for ($_i1220 = 0; $_i1220 < $_size1216; ++$_i1220)
             {
-              $elem1207 = null;
-              $xfer += $input->readString($elem1207);
-              $this->group_names []= $elem1207;
+              $elem1221 = null;
+              $xfer += $input->readString($elem1221);
+              $this->group_names []= $elem1221;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33211,9 +33275,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1208)
+          foreach ($this->part_vals as $iter1222)
           {
-            $xfer += $output->writeString($iter1208);
+            $xfer += $output->writeString($iter1222);
           }
         }
         $output->writeListEnd();
@@ -33238,9 +33302,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1209)
+          foreach ($this->group_names as $iter1223)
           {
-            $xfer += $output->writeString($iter1209);
+            $xfer += $output->writeString($iter1223);
           }
         }
         $output->writeListEnd();
@@ -33329,15 +33393,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1210 = 0;
-            $_etype1213 = 0;
-            $xfer += $input->readListBegin($_etype1213, $_size1210);
-            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
+            $_size1224 = 0;
+            $_etype1227 = 0;
+            $xfer += $input->readListBegin($_etype1227, $_size1224);
+            for ($_i1228 = 0; $_i1228 < $_size1224; ++$_i1228)
             {
-              $elem1215 = null;
-              $elem1215 = new \metastore\Partition();
-              $xfer += $elem1215->read($input);
-              $this->success []= $elem1215;
+              $elem1229 = null;
+              $elem1229 = new \metastore\Partition();
+              $xfer += $elem1229->read($input);
+              $this->success []= $elem1229;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33381,9 +33445,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1216)
+          foreach ($this->success as $iter1230)
           {
-            $xfer += $iter1216->write($output);
+            $xfer += $iter1230->write($output);
           }
         }
         $output->writeListEnd();
@@ -33504,14 +33568,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1217 = 0;
-            $_etype1220 = 0;
-            $xfer += $input->readListBegin($_etype1220, $_size1217);
-            for ($_i1221 = 0; $_i1221 < $_size1217; ++$_i1221)
+            $_size1231 = 0;
+            $_etype1234 = 0;
+            $xfer += $input->readListBegin($_etype1234, $_size1231);
+            for ($_i1235 = 0; $_i1235 < $_size1231; ++$_i1235)
             {
-              $elem1222 = null;
-              $xfer += $input->readString($elem1222);
-              $this->part_vals []= $elem1222;
+              $elem1236 = null;
+              $xfer += $input->readString($elem1236);
+              $this->part_vals []= $elem1236;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33556,9 +33620,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1223)
+          foreach ($this->part_vals as $iter1237)
           {
-            $xfer += $output->writeString($iter1223);
+            $xfer += $output->writeString($iter1237);
           }
         }
         $output->writeListEnd();
@@ -33651,14 +33715,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1224 = 0;
-            $_etype1227 = 0;
-            $xfer += $input->readListBegin($_etype1227, $_size1224);
-            for ($_i1228 = 0; $_i1228 < $_size1224; ++$_i1228)
+            $_size1238 = 0;
+            $_etype1241 = 0;
+            $xfer += $input->readListBegin($_etype1241, $_size1238);
+            for ($_i1242 = 0; $_i1242 < $_size1238; ++$_i1242)
             {
-              $elem1229 = null;
-              $xfer += $input->readString($elem1229);
-              $this->success []= $elem1229;
+              $elem1243 = null;
+              $xfer += $input->readString($elem1243);
+              $this->success []= $elem1243;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33702,9 +33766,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1230)
+          foreach ($this->success as $iter1244)
           {
-            $xfer += $output->writeString($iter1230);
+            $xfer += $output->writeString($iter1244);
           }
         }
         $output->writeListEnd();
@@ -33947,15 +34011,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1231 = 0;
-            $_etype1234 = 0;
-            $xfer += $input->readListBegin($_etype1234, $_size1231);
-            for ($_i1235 = 0; $_i1235 < $_size1231; ++$_i1235)
+            $_size1245 = 0;
+            $_etype1248 = 0;
+            $xfer += $input->readListBegin($_etype1248, $_size1245);
+            for ($_i1249 = 0; $_i1249 < $_size1245; ++$_i1249)
             {
-              $elem1236 = null;
-              $elem1236 = new \metastore\Partition();
-              $xfer += $elem1236->read($input);
-              $this->success []= $elem1236;
+              $elem1250 = null;
+              $elem1250 = new \metastore\Partition();
+              $xfer += $elem1250->read($input);
+              $this->success []= $elem1250;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33999,9 +34063,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1237)
+          foreach ($this->success as $iter1251)
           {
-            $xfer += $iter1237->write($output);
+            $xfer += $iter1251->write($output);
           }
         }
         $output->writeListEnd();
@@ -34244,15 +34308,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1238 = 0;
-            $_etype1241 = 0;
-            $xfer += $input->readListBegin($_etype1241, $_size1238);
-            for ($_i1242 = 0; $_i1242 < $_size1238; ++$_i1242)
+            $_size1252 = 0;
+            $_etype1255 = 0;
+            $xfer += $input->readListBegin($_etype1255, $_size1252);
+            for ($_i1256 = 0; $_i1256 < $_size1252; ++$_i1256)
             {
-              $elem1243 = null;
-              $elem1243 = new \metastore\PartitionSpec();
-              $xfer += $elem1243->read($input);
-              $this->success []= $elem1243;
+              $elem1257 = null;
+              $elem1257 = new \metastore\PartitionSpec();
+              $xfer += $elem1257->read($input);
+              $this->success []= $elem1257;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34296,9 +34360,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1244)
+          foreach ($this->success as $iter1258)
           {
-            $xfer += $iter1244->write($output);
+            $xfer += $iter1258->write($output);
           }
         }
         $output->writeListEnd();
@@ -34864,14 +34928,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1245 = 0;
-            $_etype1248 = 0;
-            $xfer += $input->readListBegin($_etype1248, $_size1245);
-            for ($_i1249 = 0; $_i1249 < $_size1245; ++$_i1249)
+            $_size1259 = 0;
+            $_etype1262 = 0;
+            $xfer += $input->readListBegin($_etype1262, $_size1259);
+            for ($_i1263 = 0; $_i1263 < $_size1259; ++$_i1263)
             {
-              $elem1250 = null;
-              $xfer += $input->readString($elem1250);
-              $this->names []= $elem1250;
+              $elem1264 = null;
+              $xfer += $input->readString($elem1264);
+              $this->names []= $elem1264;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34909,9 +34973,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1251)
+          foreach ($this->names as $iter1265)
           {
-            $xfer += $output->writeString($iter1251);
+            $xfer += $output->writeString($iter1265);
           }
         }
         $output->writeListEnd();
@@ -35000,15 +35064,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1252 = 0;
-            $_etype1255 = 0;
-            $xfer += $input->readListBegin($_etype1255, $_size1252);
-            for ($_i1256 = 0; $_i1256 < $_size1252; ++$_i1256)
+            $_size1266 = 0;
+            $_etype1269 = 0;
+            $xfer += $input->readListBegin($_etype1269, $_size1266);
+            for ($_i1270 = 0; $_i1270 < $_size1266; ++$_i1270)
             {
-              $elem1257 = null;
-              $elem1257 = new \metastore\Partition();
-              $xfer += $elem1257->read($input);
-              $this->success []= $elem1257;
+              $elem1271 = null;
+              $elem1271 = new \metastore\Partition();
+              $xfer += $elem1271->read($input);
+              $this->success []= $elem1271;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35052,9 +35116,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1258)
+          foreach ($this->success as $iter1272)
           {
-            $xfer += $iter1258->write($output);
+            $xfer += $iter1272->write($output);
           }
         }
         $output->writeListEnd();
@@ -35078,6 +35142,216 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
 
 }
 
+class ThriftHiveMetastore_get_partitions_by_names_req_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\GetPartitionsByNamesRequest
+   */
+  public $req = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetPartitionsByNamesRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_partitions_by_names_req_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->req = new \metastore\GetPartitionsByNamesRequest();
+            $xfer += $this->req->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_req_args');
+    if ($this->req !== null) {
+      if (!is_object($this->req)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1);
+      $xfer += $this->req->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_partitions_by_names_req_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\GetPartitionsByNamesResult
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o2 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetPartitionsByNamesResult',
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_partitions_by_names_req_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\GetPartitionsByNamesResult();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_partitions_by_names_req_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_alter_partition_args {
   static $_TSPEC;
 
@@ -35393,15 +35667,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1259 = 0;
-            $_etype1262 = 0;
-            $xfer += $input->readListBegin($_etype1262, $_size1259);
-            for ($_i1263 = 0; $_i1263 < $_size1259; ++$_i1263)
+            $_size1273 = 0;
+            $_etype1276 = 0;
+            $xfer += $input->readListBegin($_etype1276, $_size1273);
+            for ($_i1277 = 0; $_i1277 < $_size1273; ++$_i1277)
             {
-              $elem1264 = null;
-              $elem1264 = new \metastore\Partition();
-              $xfer += $elem1264->read($input);
-              $this->new_parts []= $elem1264;
+              $elem1278 = null;
+              $elem1278 = new \metastore\Partition();
+              $xfer += $elem1278->read($input);
+              $this->new_parts []= $elem1278;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35439,9 +35713,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1265)
+          foreach ($this->new_parts as $iter1279)
           {
-            $xfer += $iter1265->write($output);
+            $xfer += $iter1279->write($output);
           }
         }
         $output->writeListEnd();
@@ -35656,15 +35930,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1266 = 0;
-            $_etype1269 = 0;
-            $xfer += $input->readListBegin($_etype1269, $_size1266);
-            for ($_i1270 = 0; $_i1270 < $_size1266; ++$_i1270)
+            $_size1280 = 0;
+            $_etype1283 = 0;
+            $xfer += $input->readListBegin($_etype1283, $_size1280);
+            for ($_i1284 = 0; $_i1284 < $_size1280; ++$_i1284)
             {
-              $elem1271 = null;
-              $elem1271 = new \metastore\Partition();
-              $xfer += $elem1271->read($input);
-              $this->new_parts []= $elem1271;
+              $elem1285 = null;
+              $elem1285 = new \metastore\Partition();
+              $xfer += $elem1285->read($input);
+              $this->new_parts []= $elem1285;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35710,9 +35984,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1272)
+          foreach ($this->new_parts as $iter1286)
           {
-            $xfer += $iter1272->write($output);
+            $xfer += $iter1286->write($output);
           }
         }
         $output->writeListEnd();
@@ -36400,14 +36674,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1273 = 0;
-            $_etype1276 = 0;
-            $xfer += $input->readListBegin($_etype1276, $_size1273);
-            for ($_i1277 = 0; $_i1277 < $_size1273; ++$_i1277)
+            $_size1287 = 0;
+            $_etype1290 = 0;
+            $xfer += $input->readListBegin($_etype1290, $_size1287);
+            for ($_i1291 = 0; $_i1291 < $_size1287; ++$_i1291)
             {
-              $elem1278 = null;
-              $xfer += $input->readString($elem1278);
-              $this->part_vals []= $elem1278;
+              $elem1292 = null;
+              $xfer += $input->readString($elem1292);
+              $this->part_vals []= $elem1292;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36453,9 +36727,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1279)
+          foreach ($this->part_vals as $iter1293)
           {
-            $xfer += $output->writeString($iter1279);
+            $xfer += $output->writeString($iter1293);
           }
         }
         $output->writeListEnd();
@@ -36850,14 +37124,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1280 = 0;
-            $_etype1283 = 0;
-            $xfer += $input->readListBegin($_etype1283, $_size1280);
-            for ($_i1284 = 0; $_i1284 < $_size1280; ++$_i1284)
+            $_size1294 = 0;
+            $_etype1297 = 0;
+            $xfer += $input->readListBegin($_etype1297, $_size1294);
+            for ($_i1298 = 0; $_i1298 < $_size1294; ++$_i1298)
             {
-              $elem1285 = null;
-              $xfer += $input->readString($elem1285);
-              $this->part_vals []= $elem1285;
+              $elem1299 = null;
+              $xfer += $input->readString($elem1299);
+              $this->part_vals []= $elem1299;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36892,9 +37166,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1286)
+          foreach ($this->part_vals as $iter1300)
           {
-            $xfer += $output->writeString($iter1286);
+            $xfer += $output->writeString($iter1300);
           }
         }
         $output->writeListEnd();
@@ -37348,14 +37622,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success =

<TRUNCATED>

[11/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Signed-off-by: Sankar Hariappan <sa...@apache.org>


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

Branch: refs/heads/master
Commit: 71dfd1d11f239caf8f16bc29db0f959e566f7659
Parents: 9747083
Author: Ashutosh Bapat <ab...@cloudera.com>
Authored: Tue Jan 29 20:15:59 2019 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Tue Jan 29 20:15:59 2019 +0530

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |    1 +
 .../hive/ql/parse/TestReplicationScenarios.java |    5 +-
 .../ql/parse/TestStatsReplicationScenarios.java |  105 +-
 ...stStatsReplicationScenariosNoAutogather.java |    2 -
 .../hadoop/hive/ql/parse/WarehouseInstance.java |   14 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   54 +-
 .../events/filesystem/FSTableEvent.java         |   13 +
 .../bootstrap/load/table/LoadPartitions.java    |    1 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   92 +-
 .../hive/ql/metadata/PartitionIterable.java     |   16 +-
 .../ql/metadata/SessionHiveMetaStoreClient.java |   15 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   36 +-
 .../hive/ql/parse/repl/dump/TableExport.java    |   11 +-
 .../repl/dump/events/AddPartitionHandler.java   |    8 +
 .../repl/dump/events/AlterPartitionHandler.java |    8 +
 .../repl/dump/events/DropPartitionHandler.java  |    9 +
 .../dump/events/UpdatePartColStatHandler.java   |   30 +-
 .../dump/events/UpdateTableColStatHandler.java  |    2 +-
 .../load/message/UpdatePartColStatHandler.java  |   22 +-
 .../hadoop/hive/ql/plan/AddPartitionDesc.java   |    6 +
 .../hive/metastore/api/AbortTxnsRequest.java    |   32 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../api/AllocateTableWriteIdsRequest.java       |   68 +-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../metastore/api/AlterPartitionsRequest.java   |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/CommitTxnRequest.java    |   36 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |   32 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../hadoop/hive/metastore/api/Function.java     |   36 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/GetOpenTxnsInfoResponse.java  |   36 +-
 .../hive/metastore/api/GetOpenTxnsResponse.java |   32 +-
 .../api/GetPartitionsByNamesRequest.java        |  752 ++++
 .../api/GetPartitionsByNamesResult.java         |  443 +++
 .../metastore/api/GetPartitionsFilterSpec.java  |   32 +-
 .../api/GetPartitionsProjectionSpec.java        |   32 +-
 .../metastore/api/GetPartitionsRequest.java     |   32 +-
 .../metastore/api/GetPartitionsResponse.java    |   36 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |   32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   96 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../metastore/api/NotificationEventRequest.java |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/OpenTxnRequest.java      |   32 +-
 .../hive/metastore/api/OpenTxnsResponse.java    |   32 +-
 .../hadoop/hive/metastore/api/Partition.java    |  116 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../metastore/api/RenamePartitionRequest.java   |   32 +-
 .../hive/metastore/api/ReplLastIdInfo.java      |   32 +-
 .../api/ReplTblWriteIdStateRequest.java         |   32 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |   32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 3601 ++++++++++++------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../api/WriteNotificationLogRequest.java        |   32 +-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1660 ++++----
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1421 ++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1163 +++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  950 +++--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   45 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   66 +
 .../hive/metastore/HiveMetaStoreClient.java     |   21 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   30 +
 .../src/main/thrift/hive_metastore.thrift       |   16 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  103 +-
 .../events/UpdatePartitionColumnStatEvent.java  |   12 +-
 .../metastore/messaging/MessageBuilder.java     |    4 +-
 .../UpdatePartitionColumnStatMessage.java       |    4 +
 .../JSONUpdatePartitionColumnStatMessage.java   |   13 +-
 .../HiveMetaStoreClientPreCatalog.java          |   21 +-
 .../apache/hadoop/hive/metastore/TestStats.java |   22 +
 89 files changed, 8405 insertions(+), 4467 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 81b35a4..8404e3e 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -788,6 +788,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
             .buildUpdatePartitionColumnStatMessage(updatePartColStatEvent.getPartColStats(),
                     updatePartColStatEvent.getPartVals(),
                     updatePartColStatEvent.getPartParameters(),
+                    updatePartColStatEvent.getTableObj(),
                     updatePartColStatEvent.getValidWriteIds(), updatePartColStatEvent.getWriteId());
     NotificationEvent event = new NotificationEvent(0, now(), EventType.UPDATE_PARTITION_COLUMN_STAT.toString(),
                     msgEncoder.getSerializer().serialize(msg));

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 6e9c443..3820fab 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -423,10 +423,11 @@ public class TestReplicationScenarios {
     run("insert into table " + dbName + ".t2 partition(country='india') values ('delhi')", driver);
     dump = replDumpDb(dbName, dump.lastReplId, null, null);
 
-    //no partition task should be added as the operation is inserting into an existing partition
+    // Partition level statistics gets updated as part of the INSERT above. So we see a partition
+    // task corresponding to an ALTER_PARTITION event.
     task = getReplLoadRootTask(dbNameReplica, true, dump);
     assertEquals(true, hasMoveTask(task));
-    assertEquals(false, hasPartitionTask(task));
+    assertEquals(true, hasPartitionTask(task));
 
     loadAndVerify(dbNameReplica, dump.dumpLocation, dump.lastReplId);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java
index 8815a13..1ec4498 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenarios.java
@@ -20,9 +20,10 @@ package org.apache.hadoop.hive.ql.parse;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
-import org.apache.hadoop.hive.ql.parse.repl.PathBuilder;
 import org.apache.hadoop.hive.shims.Utils;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -112,7 +113,7 @@ public class TestStatsReplicationScenarios {
 
 
   private Map<String, String> collectStatsParams(Map<String, String> allParams) {
-    Map<String, String> statsParams = new HashMap<String, String>();
+    Map<String, String> statsParams = new HashMap<>();
     List<String> params = new ArrayList<>(StatsSetupConst.SUPPORTED_STATS);
     params.add(StatsSetupConst.COLUMN_STATS_ACCURATE);
     for (String param : params) {
@@ -125,7 +126,7 @@ public class TestStatsReplicationScenarios {
     return statsParams;
   }
 
-  private void verifyReplicatedStatsForTable(String tableName) throws Exception {
+  private void verifyReplicatedStatsForTable(String tableName) throws Throwable {
     // Test column stats
     Assert.assertEquals(primary.getTableColumnStatistics(primaryDbName, tableName),
                         replica.getTableColumnStatistics(replicatedDbName, tableName));
@@ -136,6 +137,32 @@ public class TestStatsReplicationScenarios {
     Map<String, String> pParams =
             collectStatsParams(primary.getTable(primaryDbName, tableName).getParameters());
     Assert.assertEquals(pParams, rParams);
+
+    verifyReplicatedStatsForPartitionsOfTable(tableName);
+  }
+
+  private void verifyReplicatedStatsForPartitionsOfTable(String tableName)
+          throws Throwable {
+    // Test partition level stats
+    List<Partition> pParts = primary.getAllPartitions(primaryDbName, tableName);
+
+    if (pParts == null || pParts.isEmpty()) {
+      // Not a partitioned table, nothing to verify.
+      return;
+    }
+
+    for (Partition pPart : pParts) {
+      Partition rPart = replica.getPartition(replicatedDbName, tableName,
+              pPart.getValues());
+
+      Map<String, String> rParams = collectStatsParams(rPart.getParameters());
+      Map<String, String> pParams = collectStatsParams(pPart.getParameters());
+      Assert.assertEquals(pParams, rParams);
+    }
+
+    // Test partition column stats for all partitions
+    Assert.assertEquals(primary.getAllPartitionColumnStatistics(primaryDbName, tableName),
+                        replica.getAllPartitionColumnStatistics(replicatedDbName, tableName));
   }
 
   private void verifyNoStatsReplicationForMetadataOnly(String tableName) throws Throwable {
@@ -147,7 +174,6 @@ public class TestStatsReplicationScenarios {
     // or false. Either is fine with us so don't bother checking exact values.
     Map<String, String> rParams =
             collectStatsParams(replica.getTable(replicatedDbName, tableName).getParameters());
-    List<String> params = new ArrayList<>(StatsSetupConst.SUPPORTED_STATS);
     Map<String, String> expectedFalseParams = new HashMap<>();
     Map<String, String> expectedTrueParams = new HashMap<>();
     StatsSetupConst.setStatsStateForCreateTable(expectedTrueParams,
@@ -155,24 +181,54 @@ public class TestStatsReplicationScenarios {
     StatsSetupConst.setStatsStateForCreateTable(expectedFalseParams,
             replica.getTableColNames(replicatedDbName, tableName), StatsSetupConst.FALSE);
     Assert.assertTrue(rParams.equals(expectedFalseParams) || rParams.equals(expectedTrueParams));
+
+    verifyNoPartitionStatsReplicationForMetadataOnly(tableName);
+  }
+
+  private void verifyNoPartitionStatsReplicationForMetadataOnly(String tableName) throws Throwable {
+    // Test partition level stats
+    List<Partition> pParts = primary.getAllPartitions(primaryDbName, tableName);
+
+    if (pParts == null || pParts.isEmpty()) {
+      // Not a partitioned table, nothing to verify.
+      return;
+    }
+
+    // Partitions are not replicated in metadata only replication.
+    List<Partition> rParts = replica.getAllPartitions(replicatedDbName, tableName);
+    Assert.assertTrue(rParts == null || rParts.isEmpty());
+
+    // Test partition column stats for all partitions
+    Map<String, List<ColumnStatisticsObj>> rPartColStats =
+            replica.getAllPartitionColumnStatistics(replicatedDbName, tableName);
+    for (Map.Entry<String, List<ColumnStatisticsObj>> entry: rPartColStats.entrySet()) {
+      List<ColumnStatisticsObj> colStats = entry.getValue();
+      Assert.assertTrue(colStats == null || colStats.isEmpty());
+    }
   }
 
   private List<String> createBootStrapData() throws Throwable {
+    // Unpartitioned table with data
     String simpleTableName = "sTable";
+    // partitioned table with data
     String partTableName = "pTable";
+    // Unpartitioned table without data during bootstrap and hence no stats
     String ndTableName = "ndTable";
+    // Partitioned table without data during bootstrap and hence no stats.
+    String ndPartTableName = "ndPTable";
 
     primary.run("use " + primaryDbName)
             .run("create table " + simpleTableName + " (id int)")
             .run("insert into " + simpleTableName + " values (1), (2)")
             .run("create table " + partTableName + " (place string) partitioned by (country string)")
-            .run("insert into table " + partTableName + " partition(country='india') values ('bangalore')")
-            .run("insert into table " + partTableName + " partition(country='us') values ('austin')")
-            .run("insert into table " + partTableName + " partition(country='france') values ('paris')")
-            .run("create table " + ndTableName + " (str string)");
+            .run("insert into " + partTableName + " partition(country='india') values ('bangalore')")
+            .run("insert into " + partTableName + " partition(country='us') values ('austin')")
+            .run("insert into " + partTableName + " partition(country='france') values ('paris')")
+            .run("create table " + ndTableName + " (str string)")
+            .run("create table " + ndPartTableName + " (val string) partitioned by (pk int)");
 
-    List<String> tableNames = new ArrayList<String>(Arrays.asList(simpleTableName, partTableName,
-            ndTableName));
+    List<String> tableNames = new ArrayList<>(Arrays.asList(simpleTableName, partTableName,
+            ndTableName, ndPartTableName));
 
     // Run analyze on each of the tables, if they are not being gathered automatically.
     if (!hasAutogather) {
@@ -214,7 +270,7 @@ public class TestStatsReplicationScenarios {
             .dump(primaryDbName, lastReplicationId, withClauseList);
 
     // Load, if necessary changing configuration.
-    if (parallelLoad && lastReplicationId == null) {
+    if (parallelLoad) {
       replica.hiveConf.setBoolVar(HiveConf.ConfVars.EXECPARALLEL, true);
     }
 
@@ -246,27 +302,41 @@ public class TestStatsReplicationScenarios {
   }
 
   private void createIncrementalData(List<String> tableNames) throws Throwable {
+    // Annotations for this table are same as createBootStrapData
     String simpleTableName = "sTable";
     String partTableName = "pTable";
     String ndTableName = "ndTable";
+    String ndPartTableName = "ndPTable";
 
     Assert.assertTrue(tableNames.containsAll(Arrays.asList(simpleTableName, partTableName,
-                                                         ndTableName)));
+                                                         ndTableName, ndPartTableName)));
+    // New tables created during incremental phase and thus loaded with data and stats during
+    // incremental phase.
     String incTableName = "iTable"; // New table
+    String incPartTableName = "ipTable"; // New partitioned table
 
     primary.run("use " + primaryDbName)
             .run("insert into " + simpleTableName + " values (3), (4)")
             // new data inserted into table
             .run("insert into " + ndTableName + " values ('string1'), ('string2')")
             // two partitions changed and one unchanged
-            .run("insert into table " + partTableName + " values ('india', 'pune')")
-            .run("insert into table " + partTableName + " values ('us', 'chicago')")
+            .run("insert into " + partTableName + "(country, place) values ('india', 'pune')")
+            .run("insert into " + partTableName + "(country, place) values ('us', 'chicago')")
             // new partition
-            .run("insert into table " + partTableName + " values ('australia', 'perth')")
+            .run("insert into " + partTableName + "(country, place) values ('australia', 'perth')")
             .run("create table " + incTableName + " (config string, enabled boolean)")
             .run("insert into " + incTableName + " values ('conf1', true)")
-            .run("insert into " + incTableName + " values ('conf2', false)");
+            .run("insert into " + incTableName + " values ('conf2', false)")
+            .run("insert into " + ndPartTableName + "(pk, val) values (1, 'one')")
+            .run("insert into " + ndPartTableName + "(pk, val) values (1, 'another one')")
+            .run("insert into " + ndPartTableName + "(pk, val) values (2, 'two')")
+            .run("create table " + incPartTableName +
+                    "(val string) partitioned by (tvalue boolean)")
+            .run("insert into " + incPartTableName + "(tvalue, val) values (true, 'true')")
+            .run("insert into " + incPartTableName + "(tvalue, val) values (false, 'false')");
+
     tableNames.add(incTableName);
+    tableNames.add(incPartTableName);
 
     // Run analyze on each of the tables, if they are not being gathered automatically.
     if (!hasAutogather) {
@@ -275,10 +345,9 @@ public class TestStatsReplicationScenarios {
                 .run("analyze table " + name + " compute statistics for columns");
       }
     }
-
   }
 
-  public void testStatsReplicationCommon(boolean parallelBootstrap, boolean metadataOnly) throws Throwable {
+  private void testStatsReplicationCommon(boolean parallelBootstrap, boolean metadataOnly) throws Throwable {
     List<String> tableNames = createBootStrapData();
     String lastReplicationId = dumpLoadVerify(tableNames, null, parallelBootstrap,
             metadataOnly);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenariosNoAutogather.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenariosNoAutogather.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenariosNoAutogather.java
index f58ddb8..51f8dfb 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenariosNoAutogather.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestStatsReplicationScenariosNoAutogather.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.messaging.json.gzip.GzipJSONMessageEncoder;
@@ -26,7 +25,6 @@ import org.junit.Rule;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.Assert;
 
 import java.util.HashMap;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index b272f06..e0547d4 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -404,6 +404,20 @@ public class WarehouseInstance implements Closeable {
   }
 
   /**
+   * Get statistics for given set of columns of a given table in the given database
+   * @param dbName - the database where the table resides
+   * @param tableName - tablename whose statistics are to be retrieved
+   * @param colNames - columns whose statistics is to be retrieved.
+   * @return - list of ColumnStatisticsObj objects in the order of the specified columns
+   */
+  public Map<String, List<ColumnStatisticsObj>> getAllPartitionColumnStatistics(String dbName,
+                                                                    String tableName) throws Exception {
+    List<String> colNames = new ArrayList();
+    client.getFields(dbName, tableName).forEach(fs -> colNames.add(fs.getName()));
+    return getAllPartitionColumnStatistics(dbName, tableName, colNames);
+  }
+
+  /**
    * Get statistics for given set of columns for all the partitions of a given table in the given
    * database.
    * @param dbName - the database where the table resides

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index fb35c79..4f2a116 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -269,6 +269,26 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
     return false;
   }
 
+  // Whether statistics need to be reset as part of MoveTask execution.
+  private boolean resetStatisticsProps(Table table) {
+    if (hasFollowingStatsTask()) {
+      // If there's a follow-on stats task then the stats will be correct after load, so don't
+      // need to reset the statistics.
+      return false;
+    }
+
+    if (!work.getIsInReplicationScope()) {
+      // If the load is not happening during replication and there is not follow-on stats
+      // task, stats will be inaccurate after load and so need to be reset.
+      return true;
+    }
+
+    // If we are loading a table during replication, the stats will also be replicated
+    // and hence accurate if it's a non-transactional table. For transactional table we
+    // do not replicate stats yet.
+    return AcidUtils.isTransactionalTable(table.getParameters());
+  }
+
   private final static class TaskInformation {
     public List<BucketCol> bucketCols = null;
     public List<SortCol> sortCols = null;
@@ -399,24 +419,10 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
               + " into " + tbd.getTable().getTableName());
           }
 
-          boolean resetStatistics;
-          if (hasFollowingStatsTask()) {
-            // If there's a follow-on stats task then the stats will be correct after load, so don't
-            // need to reset the statistics.
-            resetStatistics = false;
-          } else if (!work.getIsInReplicationScope()) {
-            // If the load is not happening during replication and there is not follow-on stats
-            // task, stats will be inaccurate after load and so need to be reset.
-            resetStatistics = true;
-          } else {
-            // If we are loading a table during replication, the stats will also be replicated
-            // and hence accurate if it's a non-transactional table. For transactional table we
-            // do not replicate stats yet.
-            resetStatistics = AcidUtils.isTransactionalTable(table.getParameters());
-          }
           db.loadTable(tbd.getSourcePath(), tbd.getTable().getTableName(), tbd.getLoadFileType(),
-              work.isSrcLocal(), isSkewedStoredAsDirs(tbd), isFullAcidOp, resetStatistics,
-              tbd.getWriteId(), tbd.getStmtId(), tbd.isInsertOverwrite());
+                  work.isSrcLocal(), isSkewedStoredAsDirs(tbd), isFullAcidOp,
+                  resetStatisticsProps(table), tbd.getWriteId(), tbd.getStmtId(),
+                  tbd.isInsertOverwrite());
           if (work.getOutputs() != null) {
             DDLTask.addIfAbsentByName(new WriteEntity(table,
               getWriteType(tbd, work.getLoadTableWork().getWriteType())), work.getOutputs());
@@ -513,12 +519,12 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
     }
 
     db.loadPartition(tbd.getSourcePath(), db.getTable(tbd.getTable().getTableName()),
-        tbd.getPartitionSpec(), tbd.getLoadFileType(), tbd.getInheritTableSpecs(),
-        tbd.getInheritLocation(), isSkewedStoredAsDirs(tbd), work.isSrcLocal(),
-         work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID &&
-            !tbd.isMmTable(),
-         hasFollowingStatsTask(),
-        tbd.getWriteId(), tbd.getStmtId(), tbd.isInsertOverwrite());
+            tbd.getPartitionSpec(), tbd.getLoadFileType(), tbd.getInheritTableSpecs(),
+            tbd.getInheritLocation(), isSkewedStoredAsDirs(tbd), work.isSrcLocal(),
+            work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID &&
+                    !tbd.isMmTable(),
+            resetStatisticsProps(table), tbd.getWriteId(), tbd.getStmtId(),
+            tbd.isInsertOverwrite());
     Partition partn = db.getPartition(table, tbd.getPartitionSpec(), false);
 
     // See the comment inside updatePartitionBucketSortColumns.
@@ -563,7 +569,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
             !tbd.isMmTable(),
         work.getLoadTableWork().getWriteId(),
         tbd.getStmtId(),
-        hasFollowingStatsTask(),
+        resetStatisticsProps(table),
         work.getLoadTableWork().getWriteType(),
         tbd.isInsertOverwrite());
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/events/filesystem/FSTableEvent.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/events/filesystem/FSTableEvent.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/events/filesystem/FSTableEvent.java
index d57cbd1..076165a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/events/filesystem/FSTableEvent.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/events/filesystem/FSTableEvent.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.ql.exec.repl.bootstrap.events.TableEvent;
@@ -186,6 +188,17 @@ public class FSTableEvent implements TableEvent {
             Warehouse.makePartName(tblDesc.getPartCols(), partition.getValues())).toString());
       }
       partsDesc.setReplicationSpec(replicationSpec());
+
+      // Right now, we do not have a way of associating a writeId with statistics for a table
+      // converted to a transactional table if it was non-transactional on the source. So, do not
+      // update statistics for converted tables even if available on the source.
+      if (partition.isSetColStats() && !replicationSpec().isMigratingToTxnTable()) {
+        ColumnStatistics colStats = partition.getColStats();
+        ColumnStatisticsDesc colStatsDesc = new ColumnStatisticsDesc(colStats.getStatsDesc());
+        colStatsDesc.setTableName(tblDesc.getTableName());
+        colStatsDesc.setDbName(tblDesc.getDatabaseName());
+        partDesc.setColStats(new ColumnStatistics(colStatsDesc, colStats.getStatsObj()));
+      }
       return partsDesc;
     } catch (Exception e) {
       throw new SemanticException(e);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
index e182f31..ad41276 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/bootstrap/load/table/LoadPartitions.java
@@ -313,6 +313,7 @@ public class LoadPartitions {
       loadTableWork.setInheritTableSpecs(false);
       moveWork.setLoadTableWork(loadTableWork);
     }
+    moveWork.setIsInReplicationScope(event.replicationSpec().isInReplicationScope());
 
     return TaskFactory.get(moveWork, context.hiveConf);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/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 cd59efb..9ab3a9e 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
@@ -1895,18 +1895,20 @@ public class Hive {
    *          If the source directory is LOCAL
    * @param isAcidIUDoperation
    *          true if this is an ACID operation Insert/Update/Delete operation
-   * @param hasFollowingStatsTask
-   *          true if there is a following task which updates the stats, so, this method need not update.
+   * @param resetStatistics
+   *          if true, reset the statistics. If false, do not reset statistics.
    * @param writeId write ID allocated for the current load operation
    * @param stmtId statement ID of the current load statement
    * @param isInsertOverwrite
    * @return Partition object being loaded with data
    */
   public Partition loadPartition(Path loadPath, Table tbl, Map<String, String> partSpec,
-      LoadFileType loadFileType, boolean inheritTableSpecs, boolean inheritLocation,
-      boolean isSkewedStoreAsSubdir,
-      boolean isSrcLocal, boolean isAcidIUDoperation, boolean hasFollowingStatsTask, Long writeId,
-      int stmtId, boolean isInsertOverwrite) throws HiveException {
+                                 LoadFileType loadFileType, boolean inheritTableSpecs,
+                                 boolean inheritLocation,
+                                 boolean isSkewedStoreAsSubdir,
+                                 boolean isSrcLocal, boolean isAcidIUDoperation,
+                                 boolean resetStatistics, Long writeId,
+                                 int stmtId, boolean isInsertOverwrite) throws HiveException {
 
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_PARTITION);
@@ -1923,7 +1925,7 @@ public class Hive {
     Partition newTPart = loadPartitionInternal(loadPath, tbl, partSpec, oldPart,
             loadFileType, inheritTableSpecs,
             inheritLocation, isSkewedStoreAsSubdir, isSrcLocal, isAcidIUDoperation,
-            hasFollowingStatsTask, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles);
+            resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles);
 
     AcidUtils.TableSnapshot tableSnapshot = null;
     if (isTxnTable) {
@@ -1941,7 +1943,7 @@ public class Hive {
     }
 
     if (oldPart == null) {
-      addPartitionToMetastore(newTPart, hasFollowingStatsTask, tbl, tableSnapshot);
+      addPartitionToMetastore(newTPart, resetStatistics, tbl, tableSnapshot);
       // For acid table, add the acid_write event with file list at the time of load itself. But
       // it should be done after partition is created.
       if (isTxnTable && (null != newFiles)) {
@@ -1949,7 +1951,7 @@ public class Hive {
       }
     } else {
       try {
-        setStatsPropAndAlterPartition(hasFollowingStatsTask, tbl, newTPart, tableSnapshot);
+        setStatsPropAndAlterPartition(resetStatistics, tbl, newTPart, tableSnapshot);
       } catch (TException e) {
         LOG.error(StringUtils.stringifyException(e));
         throw new HiveException(e);
@@ -1987,8 +1989,8 @@ public class Hive {
    *          If the source directory is LOCAL
    * @param isAcidIUDoperation
    *          true if this is an ACID operation Insert/Update/Delete operation
-   * @param hasFollowingStatsTask
-   *          true if there is a following task which updates the stats, so, this method need not update.
+   * @param resetStatistics
+   *          if true, reset the statistics. Do not reset statistics if false.
    * @param writeId
    *          write ID allocated for the current load operation
    * @param stmtId
@@ -2002,7 +2004,7 @@ public class Hive {
   private Partition loadPartitionInternal(Path loadPath, Table tbl, Map<String, String> partSpec,
                         Partition oldPart, LoadFileType loadFileType, boolean inheritTableSpecs,
                         boolean inheritLocation, boolean isSkewedStoreAsSubdir,
-                        boolean isSrcLocal, boolean isAcidIUDoperation, boolean hasFollowingStatsTask,
+                        boolean isSrcLocal, boolean isAcidIUDoperation, boolean resetStatistics,
                         Long writeId, int stmtId, boolean isInsertOverwrite,
                         boolean isTxnTable, List<Path> newFiles) throws HiveException {
     Path tblDataLocationPath =  tbl.getDataLocation();
@@ -2120,7 +2122,7 @@ public class Hive {
       }
 
       // column stats will be inaccurate
-      if (!hasFollowingStatsTask) {
+      if (resetStatistics) {
         StatsSetupConst.clearColumnStatsState(newTPart.getParameters());
       }
 
@@ -2168,7 +2170,7 @@ public class Hive {
     }
   }
 
-  private void addPartitionToMetastore(Partition newTPart, boolean hasFollowingStatsTask,
+  private void addPartitionToMetastore(Partition newTPart, boolean resetStatistics,
                                        Table tbl, TableSnapshot tableSnapshot) throws HiveException{
     try {
       LOG.debug("Adding new partition " + newTPart.getSpec());
@@ -2187,7 +2189,7 @@ public class Hive {
       // In that case, we want to retry with alterPartition.
       LOG.debug("Caught AlreadyExistsException, trying to alter partition instead");
       try {
-        setStatsPropAndAlterPartition(hasFollowingStatsTask, tbl, newTPart, tableSnapshot);
+        setStatsPropAndAlterPartition(resetStatistics, tbl, newTPart, tableSnapshot);
       } catch (TException e) {
         LOG.error(StringUtils.stringifyException(e));
         throw new HiveException(e);
@@ -2207,7 +2209,7 @@ public class Hive {
   }
 
   private void addPartitionsToMetastore(List<Partition> partitions,
-                                        boolean hasFollowingStatsTask, Table tbl,
+                                        boolean resetStatistics, Table tbl,
                                         List<AcidUtils.TableSnapshot> tableSnapshots)
                                         throws HiveException {
     try {
@@ -2236,7 +2238,7 @@ public class Hive {
       LOG.debug("Caught AlreadyExistsException, trying to add partitions one by one.");
       assert partitions.size() == tableSnapshots.size();
       for (int i = 0; i < partitions.size(); i++) {
-        addPartitionToMetastore(partitions.get(i), hasFollowingStatsTask, tbl,
+        addPartitionToMetastore(partitions.get(i), resetStatistics, tbl,
                 tableSnapshots.get(i));
       }
     } catch (Exception e) {
@@ -2339,10 +2341,10 @@ public class Hive {
     return;
   }
 
-  private void setStatsPropAndAlterPartition(boolean hasFollowingStatsTask, Table tbl,
-      Partition newTPart, TableSnapshot tableSnapshot) throws TException {
+  private void setStatsPropAndAlterPartition(boolean resetStatistics, Table tbl,
+                                             Partition newTPart, TableSnapshot tableSnapshot) throws TException {
     EnvironmentContext ec = new EnvironmentContext();
-    if (hasFollowingStatsTask) {
+    if (!resetStatistics) {
       ec.putToProperties(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
     }
     LOG.debug("Altering existing partition " + newTPart.getSpec());
@@ -2351,14 +2353,14 @@ public class Hive {
         tableSnapshot == null ? null : tableSnapshot.getValidWriteIdList());
   }
 
-  private void setStatsPropAndAlterPartitions(boolean hasFollowingStatsTask, Table tbl,
+  private void setStatsPropAndAlterPartitions(boolean resetStatistics, Table tbl,
                                              List<Partition> partitions,
                                              long writeId) throws TException {
     if (partitions.isEmpty()) {
       return;
     }
     EnvironmentContext ec = new EnvironmentContext();
-    if (hasFollowingStatsTask) {
+    if (!resetStatistics) {
       ec.putToProperties(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
     }
     if (LOG.isDebugEnabled()) {
@@ -2548,13 +2550,14 @@ private void constructOneLBLocationMap(FileStatus fSta,
    * @param numDP number of dynamic partitions
    * @param isAcid true if this is an ACID operation
    * @param writeId writeId, can be 0 unless isAcid == true
+   * @param resetStatistics if true, reset statistics. Do not reset statistics otherwise.
    * @return partition map details (PartitionSpec and Partition)
    * @throws HiveException
    */
   public Map<Map<String, String>, Partition> loadDynamicPartitions(final Path loadPath,
       final String tableName, final Map<String, String> partSpec, final LoadFileType loadFileType,
       final int numDP, final int numLB, final boolean isAcid, final long writeId, final int stmtId,
-      final boolean hasFollowingStatsTask, final AcidUtils.Operation operation,
+      final boolean resetStatistics, final AcidUtils.Operation operation,
       boolean isInsertOverwrite) throws HiveException {
 
     PerfLogger perfLogger = SessionState.getPerfLogger();
@@ -2630,7 +2633,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
           // load the partition
           Partition partition = loadPartitionInternal(entry.getKey(), tbl,
                   fullPartSpec, oldPartition, loadFileType, true, false, numLB > 0, false, isAcid,
-                  hasFollowingStatsTask, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles);
+                  resetStatistics, writeId, stmtId, isInsertOverwrite, isTxnTable, newFiles);
           // if the partition already existed before the loading, no need to add it again to the
           // metastore
 
@@ -2663,7 +2666,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                   + " loadFileType=" + loadFileType.toString() + ", "
                   + " listBucketingLevel=" + numLB + ", "
                   + " isAcid=" + isAcid + ", "
-                  + " hasFollowingStatsTask=" + hasFollowingStatsTask, e);
+                  + " resetStatistics=" + resetStatistics, e);
           throw e;
         }
       });
@@ -2690,7 +2693,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                       .filter(entry -> !entry.getValue().hasOldPartition)
                       .map(entry -> entry.getValue().partition)
                       .collect(Collectors.toList()),
-              hasFollowingStatsTask,
+              resetStatistics,
               tbl,
               partitionDetailsMap.entrySet()
                       .stream()
@@ -2707,7 +2710,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
         }
       }
 
-      setStatsPropAndAlterPartitions(hasFollowingStatsTask, tbl,
+      setStatsPropAndAlterPartitions(resetStatistics, tbl,
               partitionDetailsMap.entrySet().stream()
                       .filter(entry -> entry.getValue().hasOldPartition)
                       .map(entry -> entry.getValue().partition)
@@ -2731,7 +2734,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
               append("loadFileType=" + loadFileType.toString() + ", ").
               append("listBucketingLevel=" + numLB + ", ").
               append("isAcid=" + isAcid + ", ").
-              append("hasFollowingStatsTask=" + hasFollowingStatsTask);
+              append("resetStatistics=" + resetStatistics);
 
       LOG.error(logMsg.toString(), e);
       throw e;
@@ -2981,8 +2984,15 @@ private void constructOneLBLocationMap(FileStatus fSta,
             : getMSC().add_partitions(partsToAdd, addPartitionDesc.isIfNotExists(), true)) {
           out.add(new Partition(tbl, outPart));
         }
+        EnvironmentContext ec = new EnvironmentContext();
+        // In case of replication statistics is obtained from the source, so do not update those
+        // on replica. Since we are not replicating statistics for transactional tables, do not do
+        // so for a partition of a transactional table right now.
+        if (!AcidUtils.isTransactionalTable(tbl)) {
+          ec.putToProperties(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE);
+        }
         getMSC().alter_partitions(addPartitionDesc.getDbName(), addPartitionDesc.getTableName(),
-            partsToAlter, new EnvironmentContext(), validWriteIdList, writeId);
+            partsToAlter, ec, validWriteIdList, writeId);
 
         for ( org.apache.hadoop.hive.metastore.api.Partition outPart :
         getMSC().getPartitionsByNames(addPartitionDesc.getDbName(), addPartitionDesc.getTableName(),part_names)){
@@ -3033,6 +3043,9 @@ private void constructOneLBLocationMap(FileStatus fSta,
     if (addSpec.getSortCols() != null) {
       part.getSd().setSortCols(addSpec.getSortCols());
     }
+    if (addSpec.getColStats() != null) {
+      part.setColStats(addSpec.getColStats());
+    }
     return part;
   }
 
@@ -3629,6 +3642,23 @@ private void constructOneLBLocationMap(FileStatus fSta,
    */
   public List<Partition> getPartitionsByNames(Table tbl, List<String> partNames)
       throws HiveException {
+    return getPartitionsByNames(tbl, partNames, false);
+  }
+
+  /**
+   * Get all partitions of the table that matches the list of given partition names.
+   *
+   * @param tbl
+   *          object for which partition is needed. Must be partitioned.
+   * @param partNames
+   *          list of partition names
+   * @param getColStats
+   *          if true, Partition object includes column statistics for that partition.
+   * @return list of partition objects
+   * @throws HiveException
+   */
+  public List<Partition> getPartitionsByNames(Table tbl, List<String> partNames, boolean getColStats)
+      throws HiveException {
 
     if (!tbl.isPartitioned()) {
       throw new HiveException(ErrorMsg.TABLE_NOT_PARTITIONED, tbl.getTableName());
@@ -3644,7 +3674,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       for (int i = 0; i < nBatches; ++i) {
         List<org.apache.hadoop.hive.metastore.api.Partition> tParts =
           getMSC().getPartitionsByNames(tbl.getDbName(), tbl.getTableName(),
-          partNames.subList(i*batchSize, (i+1)*batchSize));
+            partNames.subList(i*batchSize, (i+1)*batchSize), getColStats);
         if (tParts != null) {
           for (org.apache.hadoop.hive.metastore.api.Partition tpart: tParts) {
             partitions.add(new Partition(tbl, tpart));
@@ -3655,7 +3685,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       if (nParts > nBatches * batchSize) {
         List<org.apache.hadoop.hive.metastore.api.Partition> tParts =
           getMSC().getPartitionsByNames(tbl.getDbName(), tbl.getTableName(),
-          partNames.subList(nBatches*batchSize, nParts));
+            partNames.subList(nBatches*batchSize, nParts), getColStats);
         if (tParts != null) {
           for (org.apache.hadoop.hive.metastore.api.Partition tpart: tParts) {
             partitions.add(new Partition(tbl, tpart));

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
index 22ba14c..6418bd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/PartitionIterable.java
@@ -24,7 +24,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-
 /**
  * PartitionIterable - effectively a lazy Iterable<Partition>
  *
@@ -100,7 +99,7 @@ public class PartitionIterable implements Iterable<Partition> {
           batch_counter++;
         }
         try {
-          batchIter = db.getPartitionsByNames(table,nameBatch).iterator();
+          batchIter = db.getPartitionsByNames(table, nameBatch, getColStats).iterator();
         } catch (HiveException e) {
           throw new RuntimeException(e);
         }
@@ -130,6 +129,7 @@ public class PartitionIterable implements Iterable<Partition> {
   private Map<String, String> partialPartitionSpec = null;
   private List<String> partitionNames = null;
   private int batch_size;
+  private boolean getColStats = false;
 
   /**
    * Dummy constructor, which simply acts as an iterator on an already-present
@@ -146,12 +146,22 @@ public class PartitionIterable implements Iterable<Partition> {
    * a Hive object and a table object, and a partial partition spec.
    */
   public PartitionIterable(Hive db, Table table, Map<String, String> partialPartitionSpec,
-      int batch_size) throws HiveException {
+                           int batch_size) throws HiveException {
+    this(db, table, partialPartitionSpec, batch_size, false);
+  }
+
+  /**
+   * Primary constructor that fetches all partitions in a given table, given
+   * a Hive object and a table object, and a partial partition spec.
+   */
+  public PartitionIterable(Hive db, Table table, Map<String, String> partialPartitionSpec,
+                           int batch_size, boolean getColStats) throws HiveException {
     this.currType = Type.LAZY_FETCH_PARTITIONS;
     this.db = db;
     this.table = table;
     this.partialPartitionSpec = partialPartitionSpec;
     this.batch_size = batch_size;
+    this.getColStats = getColStats;
 
     if (this.partialPartitionSpec == null){
       partitionNames = db.getPartitionNames(

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 83cb3ea..410868c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -1102,16 +1102,27 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     }
     return matchedParts;
   }
+
   /**
    * partNames are like "p=1/q=2" type strings.  The values (RHS of =) are escaped.
    */
   @Override
   public List<Partition> getPartitionsByNames(String db_name, String tblName,
-      List<String> partNames) throws TException {
+                                              List<String> partNames) throws TException {
+    return getPartitionsByNames(db_name, tblName, partNames, false);
+  }
+
+  /**
+   * partNames are like "p=1/q=2" type strings.  The values (RHS of =) are escaped.
+   */
+  @Override
+  public List<Partition> getPartitionsByNames(String db_name, String tblName,
+                                              List<String> partNames, boolean getColStats)
+          throws TException {
     org.apache.hadoop.hive.metastore.api.Table table = getTempTable(db_name, tblName);
     if (table == null) {
       //(assume) not a temp table - Try underlying client
-      return super.getPartitionsByNames(db_name, tblName, partNames);
+      return super.getPartitionsByNames(db_name, tblName, partNames, getColStats);
     }
     TempTable tt = getTempTable(table);
     if(tt == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index 6102339..8242b47 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -341,7 +341,11 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           getBaseAddPartitionDescFromPartition(fromPath, dbname, tblDesc, partition,
               replicationSpec, x.getConf());
       if (inReplicationScope){
-        StatsSetupConst.setBasicStatsState(partsDesc.getPartition(0).getPartParams(), StatsSetupConst.FALSE);
+        // Statistics for a non-transactional table will be replicated separately. Don't bother
+        // with it here.
+        if (TxnUtils.isTransactionalTable(tblDesc.getTblProps())) {
+          StatsSetupConst.setBasicStatsState(partsDesc.getPartition(0).getPartParams(), StatsSetupConst.FALSE);
+        }
       }
       partitionDescs.add(partsDesc);
     }
@@ -1190,22 +1194,26 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       Task t = createTableTask(tblDesc, x);
       table = createNewTableMetadataObject(tblDesc, true);
 
-      if (!replicationSpec.isMetadataOnly()) {
-        if (isPartitioned(tblDesc)) {
-          for (AddPartitionDesc addPartitionDesc : partitionDescs) {
-            addPartitionDesc.setReplicationSpec(replicationSpec);
+      if (isPartitioned(tblDesc)) {
+        for (AddPartitionDesc addPartitionDesc : partitionDescs) {
+          addPartitionDesc.setReplicationSpec(replicationSpec);
+          if (!replicationSpec.isMetadataOnly()) {
             t.addDependentTask(
-                addSinglePartition(tblDesc, table, wh, addPartitionDesc, replicationSpec, x,
-                    writeId, stmtId));
-            if (updatedMetadata != null) {
-              updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
-                      addPartitionDesc.getPartition(0).getPartSpec());
-            }
+                    addSinglePartition(tblDesc, table, wh, addPartitionDesc, replicationSpec, x,
+                            writeId, stmtId));
+          } else {
+            t.addDependentTask(alterSinglePartition(tblDesc, table, wh, addPartitionDesc,
+                    replicationSpec, null, x));
+          }
+          if (updatedMetadata != null) {
+            updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                    addPartitionDesc.getPartition(0).getPartSpec());
           }
-        } else {
-          x.getLOG().debug("adding dependent CopyWork/MoveWork for table");
-          t.addDependentTask(loadTable(fromURI, table, replicationSpec.isReplace(), new Path(tblDesc.getLocation()), replicationSpec, x, writeId, stmtId));
         }
+      } else if (!replicationSpec.isMetadataOnly()) {
+        x.getLOG().debug("adding dependent CopyWork/MoveWork for table");
+        t.addDependentTask(loadTable(fromURI, table, replicationSpec.isReplace(),
+                new Path(tblDesc.getLocation()), replicationSpec, x, writeId, stmtId));
       }
 
       if (dropTblTask != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java
index adc9446..1eee3fd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/TableExport.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -108,8 +109,16 @@ public class TableExport {
           if (replicationSpec.isMetadataOnly()) {
             return null;
           } else {
+            // For transactional tables, we do not replicate statistics right now, so don't
+            // include statistics in Partition object as well.
+            boolean getColStats;
+            if (AcidUtils.isTransactionalTable(tableSpec.tableHandle)) {
+              getColStats = false;
+            } else {
+              getColStats = true;
+            }
             return new PartitionIterable(db, tableSpec.tableHandle, null, conf.getIntVar(
-                HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
+                HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_MAX), getColStats);
           }
         } else {
           // PARTITIONS specified - partitions inside tableSpec

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AddPartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AddPartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AddPartitionHandler.java
index 0b7f910..415e954 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AddPartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AddPartitionHandler.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
@@ -51,6 +52,13 @@ class AddPartitionHandler extends AbstractEventHandler {
   public void handle(Context withinContext) throws Exception {
     LOG.info("Processing#{} ADD_PARTITION message : {}", fromEventId(), eventMessageAsJSON);
 
+    // We do not dump partitions during metadata only bootstrap dump (See TableExport
+    // .getPartitions(), for bootstrap dump we pass tableSpec with TABLE_ONLY set.). So don't
+    // dump partition related events for metadata-only dump.
+    if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      return;
+    }
+
     AddPartitionMessage apm = (AddPartitionMessage) eventMessage;
     org.apache.hadoop.hive.metastore.api.Table tobj = apm.getTableObj();
     if (tobj == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterPartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterPartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterPartitionHandler.java
index d81408e..1b91e3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterPartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/AlterPartitionHandler.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.ql.metadata.Partition;
@@ -91,6 +92,13 @@ class AlterPartitionHandler extends AbstractEventHandler<AlterPartitionMessage>
   public void handle(Context withinContext) throws Exception {
     LOG.info("Processing#{} ALTER_PARTITION message : {}", fromEventId(), eventMessageAsJSON);
 
+    // We do not dump partitions during metadata only bootstrap dump (See TableExport
+    // .getPartitions(), for bootstrap dump we pass tableSpec with TABLE_ONLY set.). So don't
+    // dump partition related events for metadata-only dump.
+    if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      return;
+    }
+
     Table qlMdTable = new Table(tableObject);
     if (!Utils.shouldReplicate(withinContext.replicationSpec, qlMdTable, withinContext.hiveConf)) {
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropPartitionHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropPartitionHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropPartitionHandler.java
index e2a40d2..272f5ce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropPartitionHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/DropPartitionHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 
 import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage;
@@ -38,6 +39,14 @@ class DropPartitionHandler extends AbstractEventHandler<DropPartitionMessage> {
   @Override
   public void handle(Context withinContext) throws Exception {
     LOG.info("Processing#{} DROP_PARTITION message : {}", fromEventId(), eventMessageAsJSON);
+
+    // We do not dump partitions during metadata only bootstrap dump (See TableExport
+    // .getPartitions(), for bootstrap dump we pass tableSpec with TABLE_ONLY set.). So don't
+    // dump partition related events for metadata-only dump.
+    if (withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      return;
+    }
+
     DumpMetaData dmd = withinContext.createDmd(this);
     dmd.setPayload(eventMessageAsJSON);
     dmd.write();

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdatePartColStatHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdatePartColStatHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdatePartColStatHandler.java
index 332005b..f3f00c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdatePartColStatHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdatePartColStatHandler.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.UpdatePartitionColumnStatMessage;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
+import org.apache.hadoop.hive.ql.parse.repl.dump.Utils;
 import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
 
 class UpdatePartColStatHandler extends AbstractEventHandler<UpdatePartitionColumnStatMessage> {
@@ -35,7 +38,32 @@ class UpdatePartColStatHandler extends AbstractEventHandler<UpdatePartitionColum
 
   @Override
   public void handle(Context withinContext) throws Exception {
-    LOG.info("Processing#{} UpdateTableColumnStat message : {}", fromEventId(), eventMessageAsJSON);
+    LOG.info("Processing#{} UpdatePartitionTableColumnStat message : {}", fromEventId(),
+            eventMessageAsJSON);
+
+    org.apache.hadoop.hive.metastore.api.Table tableObj = eventMessage.getTableObject();
+    if (tableObj == null) {
+      LOG.debug("Event#{} was an event of type {} with no table listed", fromEventId(),
+              event.getEventType());
+      return;
+    }
+
+    // Statistics without any data does not make sense.
+    if (withinContext.replicationSpec.isMetadataOnly()) {
+      return;
+    }
+
+    // For now we do not dump statistics for a transactional table since replicating the same is
+    // not supported.
+    if (AcidUtils.isTransactionalTable(tableObj)) {
+      return;
+    }
+
+    if (!Utils.shouldReplicate(withinContext.replicationSpec, new Table(tableObj),
+                              withinContext.hiveConf)) {
+      return;
+    }
+
     DumpMetaData dmd = withinContext.createDmd(this);
     dmd.setPayload(eventMessageAsJSON);
     dmd.write();

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdateTableColStatHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdateTableColStatHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdateTableColStatHandler.java
index e50a2bc..bd8182d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdateTableColStatHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/UpdateTableColStatHandler.java
@@ -36,6 +36,7 @@ class UpdateTableColStatHandler extends AbstractEventHandler<UpdateTableColumnSt
 
   @Override
   public void handle(Context withinContext) throws Exception {
+    LOG.info("Processing#{} UpdateTableColumnStat message : {}", fromEventId(), eventMessageAsJSON);
     Table qlMdTable = new Table(eventMessage.getTableObject());
     if (!Utils.shouldReplicate(withinContext.replicationSpec, qlMdTable, withinContext.hiveConf)) {
       return;
@@ -50,7 +51,6 @@ class UpdateTableColStatHandler extends AbstractEventHandler<UpdateTableColumnSt
       return;
     }
 
-    LOG.info("Processing#{} UpdateTableColumnStat message : {}", fromEventId(), eventMessageAsJSON);
     DumpMetaData dmd = withinContext.createDmd(this);
     dmd.setPayload(eventMessageAsJSON);
     dmd.write();

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/UpdatePartColStatHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/UpdatePartColStatHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/UpdatePartColStatHandler.java
index 4ba2ac4..02e938e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/UpdatePartColStatHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/UpdatePartColStatHandler.java
@@ -17,10 +17,13 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.load.message;
 
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.messaging.UpdatePartitionColumnStatMessage;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork;
+import org.apache.hadoop.hive.ql.plan.ColumnStatsUpdateWork;
 
 import java.io.Serializable;
 import java.util.Collections;
@@ -34,10 +37,21 @@ public class UpdatePartColStatHandler extends AbstractMessageHandler {
   @Override
   public List<Task<? extends Serializable>> handle(Context context)
       throws SemanticException {
-    context.log.info("Replication of partition stat update event is not supported yet");
+    UpdatePartitionColumnStatMessage upcsm =
+            deserializer.getUpdatePartitionColumnStatMessage(context.dmd.getPayload());
+
+    // Update tablename and database name in the statistics object
+    ColumnStatistics colStats = upcsm.getColumnStatistics();
+    ColumnStatisticsDesc colStatsDesc = colStats.getStatsDesc();
+    if (!context.isTableNameEmpty()) {
+      colStatsDesc.setTableName(context.tableName);
+    }
     if (!context.isDbNameEmpty()) {
-      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+      colStatsDesc.setDbName(context.dbName);
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName,
+                  null);
     }
-    return Collections.singletonList(TaskFactory.get(new DependencyCollectionWork(), context.hiveConf));
+    return Collections.singletonList(TaskFactory.get(new ColumnStatsUpdateWork(colStats),
+            context.hiveConf));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/ql/src/java/org/apache/hadoop/hive/ql/plan/AddPartitionDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AddPartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AddPartitionDesc.java
index d3a87f9..26cb217 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AddPartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AddPartitionDesc.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 
 /**
@@ -58,6 +59,7 @@ public class AddPartitionDesc extends DDLDesc implements Serializable {
     Map<String, String> serdeParams = null;
     List<String> bucketCols = null;
     List<Order> sortCols = null;
+    ColumnStatistics colStats = null;
 
     public Map<String, String> getPartSpec() {
       return partSpec;
@@ -145,6 +147,10 @@ public class AddPartitionDesc extends DDLDesc implements Serializable {
     public void setOutputFormat(String outputFormat) {
       this.outputFormat = outputFormat;
     }
+
+    public ColumnStatistics getColStats() { return colStats; }
+
+    public void setColStats(ColumnStatistics colStats) { this.colStats = colStats; }
   }
 
   private static final long serialVersionUID = 1L;

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
index 7dcfc17..9661778 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
-                struct.txn_ids = new ArrayList<Long>(_list594.size);
-                long _elem595;
-                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list610.size);
+                long _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem595 = iprot.readI64();
-                  struct.txn_ids.add(_elem595);
+                  _elem611 = iprot.readI64();
+                  struct.txn_ids.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
-          for (long _iter597 : struct.txn_ids)
+          for (long _iter613 : struct.txn_ids)
           {
-            oprot.writeI64(_iter597);
+            oprot.writeI64(_iter613);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txn_ids.size());
-        for (long _iter598 : struct.txn_ids)
+        for (long _iter614 : struct.txn_ids)
         {
-          oprot.writeI64(_iter598);
+          oprot.writeI64(_iter614);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.txn_ids = new ArrayList<Long>(_list599.size);
-        long _elem600;
-        for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list615.size);
+        long _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem600 = iprot.readI64();
-          struct.txn_ids.add(_elem600);
+          _elem616 = iprot.readI64();
+          struct.txn_ids.add(_elem616);
         }
       }
       struct.setTxn_idsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 9c33229..d678c02 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list724.size);
-                String _elem725;
-                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list740.size);
+                String _elem741;
+                for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                 {
-                  _elem725 = iprot.readString();
-                  struct.partitionnames.add(_elem725);
+                  _elem741 = iprot.readString();
+                  struct.partitionnames.add(_elem741);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter727 : struct.partitionnames)
+          for (String _iter743 : struct.partitionnames)
           {
-            oprot.writeString(_iter727);
+            oprot.writeString(_iter743);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter728 : struct.partitionnames)
+        for (String _iter744 : struct.partitionnames)
         {
-          oprot.writeString(_iter728);
+          oprot.writeString(_iter744);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list729.size);
-        String _elem730;
-        for (int _i731 = 0; _i731 < _list729.size; ++_i731)
+        org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list745.size);
+        String _elem746;
+        for (int _i747 = 0; _i747 < _list745.size; ++_i747)
         {
-          _elem730 = iprot.readString();
-          struct.partitionnames.add(_elem730);
+          _elem746 = iprot.readString();
+          struct.partitionnames.add(_elem746);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index d05e7ba..3dd36da 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -716,13 +716,13 @@ import org.slf4j.LoggerFactory;
           case 3: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                struct.txnIds = new ArrayList<Long>(_list650.size);
-                long _elem651;
-                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                struct.txnIds = new ArrayList<Long>(_list666.size);
+                long _elem667;
+                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                 {
-                  _elem651 = iprot.readI64();
-                  struct.txnIds.add(_elem651);
+                  _elem667 = iprot.readI64();
+                  struct.txnIds.add(_elem667);
                 }
                 iprot.readListEnd();
               }
@@ -742,14 +742,14 @@ import org.slf4j.LoggerFactory;
           case 5: // SRC_TXN_TO_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list653 = iprot.readListBegin();
-                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list653.size);
-                TxnToWriteId _elem654;
-                for (int _i655 = 0; _i655 < _list653.size; ++_i655)
+                org.apache.thrift.protocol.TList _list669 = iprot.readListBegin();
+                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list669.size);
+                TxnToWriteId _elem670;
+                for (int _i671 = 0; _i671 < _list669.size; ++_i671)
                 {
-                  _elem654 = new TxnToWriteId();
-                  _elem654.read(iprot);
-                  struct.srcTxnToWriteIdList.add(_elem654);
+                  _elem670 = new TxnToWriteId();
+                  _elem670.read(iprot);
+                  struct.srcTxnToWriteIdList.add(_elem670);
                 }
                 iprot.readListEnd();
               }
@@ -786,9 +786,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-            for (long _iter656 : struct.txnIds)
+            for (long _iter672 : struct.txnIds)
             {
-              oprot.writeI64(_iter656);
+              oprot.writeI64(_iter672);
             }
             oprot.writeListEnd();
           }
@@ -807,9 +807,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SRC_TXN_TO_WRITE_ID_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.srcTxnToWriteIdList.size()));
-            for (TxnToWriteId _iter657 : struct.srcTxnToWriteIdList)
+            for (TxnToWriteId _iter673 : struct.srcTxnToWriteIdList)
             {
-              _iter657.write(oprot);
+              _iter673.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -849,9 +849,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnIds()) {
         {
           oprot.writeI32(struct.txnIds.size());
-          for (long _iter658 : struct.txnIds)
+          for (long _iter674 : struct.txnIds)
           {
-            oprot.writeI64(_iter658);
+            oprot.writeI64(_iter674);
           }
         }
       }
@@ -861,9 +861,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSrcTxnToWriteIdList()) {
         {
           oprot.writeI32(struct.srcTxnToWriteIdList.size());
-          for (TxnToWriteId _iter659 : struct.srcTxnToWriteIdList)
+          for (TxnToWriteId _iter675 : struct.srcTxnToWriteIdList)
           {
-            _iter659.write(oprot);
+            _iter675.write(oprot);
           }
         }
       }
@@ -879,13 +879,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list660 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.txnIds = new ArrayList<Long>(_list660.size);
-          long _elem661;
-          for (int _i662 = 0; _i662 < _list660.size; ++_i662)
+          org.apache.thrift.protocol.TList _list676 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.txnIds = new ArrayList<Long>(_list676.size);
+          long _elem677;
+          for (int _i678 = 0; _i678 < _list676.size; ++_i678)
           {
-            _elem661 = iprot.readI64();
-            struct.txnIds.add(_elem661);
+            _elem677 = iprot.readI64();
+            struct.txnIds.add(_elem677);
           }
         }
         struct.setTxnIdsIsSet(true);
@@ -896,14 +896,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list663.size);
-          TxnToWriteId _elem664;
-          for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+          org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list679.size);
+          TxnToWriteId _elem680;
+          for (int _i681 = 0; _i681 < _list679.size; ++_i681)
           {
-            _elem664 = new TxnToWriteId();
-            _elem664.read(iprot);
-            struct.srcTxnToWriteIdList.add(_elem664);
+            _elem680 = new TxnToWriteId();
+            _elem680.read(iprot);
+            struct.srcTxnToWriteIdList.add(_elem680);
           }
         }
         struct.setSrcTxnToWriteIdListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index 193179b..dd73a69 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list666.size);
-                TxnToWriteId _elem667;
-                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
+                org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list682.size);
+                TxnToWriteId _elem683;
+                for (int _i684 = 0; _i684 < _list682.size; ++_i684)
                 {
-                  _elem667 = new TxnToWriteId();
-                  _elem667.read(iprot);
-                  struct.txnToWriteIds.add(_elem667);
+                  _elem683 = new TxnToWriteId();
+                  _elem683.read(iprot);
+                  struct.txnToWriteIds.add(_elem683);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter669 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter685 : struct.txnToWriteIds)
           {
-            _iter669.write(oprot);
+            _iter685.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter670 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter686 : struct.txnToWriteIds)
         {
-          _iter670.write(oprot);
+          _iter686.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list671.size);
-        TxnToWriteId _elem672;
-        for (int _i673 = 0; _i673 < _list671.size; ++_i673)
+        org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list687.size);
+        TxnToWriteId _elem688;
+        for (int _i689 = 0; _i689 < _list687.size; ++_i689)
         {
-          _elem672 = new TxnToWriteId();
-          _elem672.read(iprot);
-          struct.txnToWriteIds.add(_elem672);
+          _elem688 = new TxnToWriteId();
+          _elem688.read(iprot);
+          struct.txnToWriteIds.add(_elem688);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);


[09/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java
new file mode 100644
index 0000000..91199e5
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesRequest.java
@@ -0,0 +1,752 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetPartitionsByNamesRequest implements org.apache.thrift.TBase<GetPartitionsByNamesRequest, GetPartitionsByNamesRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsByNamesRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsByNamesRequest");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("db_name", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tbl_name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("names", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField GET_COL_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("get_col_stats", org.apache.thrift.protocol.TType.BOOL, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetPartitionsByNamesRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsByNamesRequestTupleSchemeFactory());
+  }
+
+  private String db_name; // required
+  private String tbl_name; // required
+  private List<String> names; // optional
+  private boolean get_col_stats; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    DB_NAME((short)1, "db_name"),
+    TBL_NAME((short)2, "tbl_name"),
+    NAMES((short)3, "names"),
+    GET_COL_STATS((short)4, "get_col_stats");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DB_NAME
+          return DB_NAME;
+        case 2: // TBL_NAME
+          return TBL_NAME;
+        case 3: // NAMES
+          return NAMES;
+        case 4: // GET_COL_STATS
+          return GET_COL_STATS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __GET_COL_STATS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAMES,_Fields.GET_COL_STATS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("db_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tbl_name", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAMES, new org.apache.thrift.meta_data.FieldMetaData("names", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.GET_COL_STATS, new org.apache.thrift.meta_data.FieldMetaData("get_col_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsByNamesRequest.class, metaDataMap);
+  }
+
+  public GetPartitionsByNamesRequest() {
+  }
+
+  public GetPartitionsByNamesRequest(
+    String db_name,
+    String tbl_name)
+  {
+    this();
+    this.db_name = db_name;
+    this.tbl_name = tbl_name;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsByNamesRequest(GetPartitionsByNamesRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetDb_name()) {
+      this.db_name = other.db_name;
+    }
+    if (other.isSetTbl_name()) {
+      this.tbl_name = other.tbl_name;
+    }
+    if (other.isSetNames()) {
+      List<String> __this__names = new ArrayList<String>(other.names);
+      this.names = __this__names;
+    }
+    this.get_col_stats = other.get_col_stats;
+  }
+
+  public GetPartitionsByNamesRequest deepCopy() {
+    return new GetPartitionsByNamesRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.db_name = null;
+    this.tbl_name = null;
+    this.names = null;
+    setGet_col_statsIsSet(false);
+    this.get_col_stats = false;
+  }
+
+  public String getDb_name() {
+    return this.db_name;
+  }
+
+  public void setDb_name(String db_name) {
+    this.db_name = db_name;
+  }
+
+  public void unsetDb_name() {
+    this.db_name = null;
+  }
+
+  /** Returns true if field db_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetDb_name() {
+    return this.db_name != null;
+  }
+
+  public void setDb_nameIsSet(boolean value) {
+    if (!value) {
+      this.db_name = null;
+    }
+  }
+
+  public String getTbl_name() {
+    return this.tbl_name;
+  }
+
+  public void setTbl_name(String tbl_name) {
+    this.tbl_name = tbl_name;
+  }
+
+  public void unsetTbl_name() {
+    this.tbl_name = null;
+  }
+
+  /** Returns true if field tbl_name is set (has been assigned a value) and false otherwise */
+  public boolean isSetTbl_name() {
+    return this.tbl_name != null;
+  }
+
+  public void setTbl_nameIsSet(boolean value) {
+    if (!value) {
+      this.tbl_name = null;
+    }
+  }
+
+  public int getNamesSize() {
+    return (this.names == null) ? 0 : this.names.size();
+  }
+
+  public java.util.Iterator<String> getNamesIterator() {
+    return (this.names == null) ? null : this.names.iterator();
+  }
+
+  public void addToNames(String elem) {
+    if (this.names == null) {
+      this.names = new ArrayList<String>();
+    }
+    this.names.add(elem);
+  }
+
+  public List<String> getNames() {
+    return this.names;
+  }
+
+  public void setNames(List<String> names) {
+    this.names = names;
+  }
+
+  public void unsetNames() {
+    this.names = null;
+  }
+
+  /** Returns true if field names is set (has been assigned a value) and false otherwise */
+  public boolean isSetNames() {
+    return this.names != null;
+  }
+
+  public void setNamesIsSet(boolean value) {
+    if (!value) {
+      this.names = null;
+    }
+  }
+
+  public boolean isGet_col_stats() {
+    return this.get_col_stats;
+  }
+
+  public void setGet_col_stats(boolean get_col_stats) {
+    this.get_col_stats = get_col_stats;
+    setGet_col_statsIsSet(true);
+  }
+
+  public void unsetGet_col_stats() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __GET_COL_STATS_ISSET_ID);
+  }
+
+  /** Returns true if field get_col_stats is set (has been assigned a value) and false otherwise */
+  public boolean isSetGet_col_stats() {
+    return EncodingUtils.testBit(__isset_bitfield, __GET_COL_STATS_ISSET_ID);
+  }
+
+  public void setGet_col_statsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __GET_COL_STATS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDb_name();
+      } else {
+        setDb_name((String)value);
+      }
+      break;
+
+    case TBL_NAME:
+      if (value == null) {
+        unsetTbl_name();
+      } else {
+        setTbl_name((String)value);
+      }
+      break;
+
+    case NAMES:
+      if (value == null) {
+        unsetNames();
+      } else {
+        setNames((List<String>)value);
+      }
+      break;
+
+    case GET_COL_STATS:
+      if (value == null) {
+        unsetGet_col_stats();
+      } else {
+        setGet_col_stats((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDb_name();
+
+    case TBL_NAME:
+      return getTbl_name();
+
+    case NAMES:
+      return getNames();
+
+    case GET_COL_STATS:
+      return isGet_col_stats();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DB_NAME:
+      return isSetDb_name();
+    case TBL_NAME:
+      return isSetTbl_name();
+    case NAMES:
+      return isSetNames();
+    case GET_COL_STATS:
+      return isSetGet_col_stats();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsByNamesRequest)
+      return this.equals((GetPartitionsByNamesRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsByNamesRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_db_name = true && this.isSetDb_name();
+    boolean that_present_db_name = true && that.isSetDb_name();
+    if (this_present_db_name || that_present_db_name) {
+      if (!(this_present_db_name && that_present_db_name))
+        return false;
+      if (!this.db_name.equals(that.db_name))
+        return false;
+    }
+
+    boolean this_present_tbl_name = true && this.isSetTbl_name();
+    boolean that_present_tbl_name = true && that.isSetTbl_name();
+    if (this_present_tbl_name || that_present_tbl_name) {
+      if (!(this_present_tbl_name && that_present_tbl_name))
+        return false;
+      if (!this.tbl_name.equals(that.tbl_name))
+        return false;
+    }
+
+    boolean this_present_names = true && this.isSetNames();
+    boolean that_present_names = true && that.isSetNames();
+    if (this_present_names || that_present_names) {
+      if (!(this_present_names && that_present_names))
+        return false;
+      if (!this.names.equals(that.names))
+        return false;
+    }
+
+    boolean this_present_get_col_stats = true && this.isSetGet_col_stats();
+    boolean that_present_get_col_stats = true && that.isSetGet_col_stats();
+    if (this_present_get_col_stats || that_present_get_col_stats) {
+      if (!(this_present_get_col_stats && that_present_get_col_stats))
+        return false;
+      if (this.get_col_stats != that.get_col_stats)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_db_name = true && (isSetDb_name());
+    list.add(present_db_name);
+    if (present_db_name)
+      list.add(db_name);
+
+    boolean present_tbl_name = true && (isSetTbl_name());
+    list.add(present_tbl_name);
+    if (present_tbl_name)
+      list.add(tbl_name);
+
+    boolean present_names = true && (isSetNames());
+    list.add(present_names);
+    if (present_names)
+      list.add(names);
+
+    boolean present_get_col_stats = true && (isSetGet_col_stats());
+    list.add(present_get_col_stats);
+    if (present_get_col_stats)
+      list.add(get_col_stats);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsByNamesRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDb_name()).compareTo(other.isSetDb_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDb_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db_name, other.db_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTbl_name()).compareTo(other.isSetTbl_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTbl_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tbl_name, other.tbl_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetNames()).compareTo(other.isSetNames());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNames()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.names, other.names);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetGet_col_stats()).compareTo(other.isSetGet_col_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetGet_col_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.get_col_stats, other.get_col_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetPartitionsByNamesRequest(");
+    boolean first = true;
+
+    sb.append("db_name:");
+    if (this.db_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.db_name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tbl_name:");
+    if (this.tbl_name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tbl_name);
+    }
+    first = false;
+    if (isSetNames()) {
+      if (!first) sb.append(", ");
+      sb.append("names:");
+      if (this.names == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.names);
+      }
+      first = false;
+    }
+    if (isSetGet_col_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("get_col_stats:");
+      sb.append(this.get_col_stats);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDb_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'db_name' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTbl_name()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tbl_name' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetPartitionsByNamesRequestStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsByNamesRequestStandardScheme getScheme() {
+      return new GetPartitionsByNamesRequestStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsByNamesRequestStandardScheme extends StandardScheme<GetPartitionsByNamesRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsByNamesRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.db_name = iprot.readString();
+              struct.setDb_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tbl_name = iprot.readString();
+              struct.setTbl_nameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NAMES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list554 = iprot.readListBegin();
+                struct.names = new ArrayList<String>(_list554.size);
+                String _elem555;
+                for (int _i556 = 0; _i556 < _list554.size; ++_i556)
+                {
+                  _elem555 = iprot.readString();
+                  struct.names.add(_elem555);
+                }
+                iprot.readListEnd();
+              }
+              struct.setNamesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // GET_COL_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.get_col_stats = iprot.readBool();
+              struct.setGet_col_statsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsByNamesRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.db_name != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.db_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tbl_name != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tbl_name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.names != null) {
+        if (struct.isSetNames()) {
+          oprot.writeFieldBegin(NAMES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.names.size()));
+            for (String _iter557 : struct.names)
+            {
+              oprot.writeString(_iter557);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetGet_col_stats()) {
+        oprot.writeFieldBegin(GET_COL_STATS_FIELD_DESC);
+        oprot.writeBool(struct.get_col_stats);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsByNamesRequestTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsByNamesRequestTupleScheme getScheme() {
+      return new GetPartitionsByNamesRequestTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsByNamesRequestTupleScheme extends TupleScheme<GetPartitionsByNamesRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNamesRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.db_name);
+      oprot.writeString(struct.tbl_name);
+      BitSet optionals = new BitSet();
+      if (struct.isSetNames()) {
+        optionals.set(0);
+      }
+      if (struct.isSetGet_col_stats()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetNames()) {
+        {
+          oprot.writeI32(struct.names.size());
+          for (String _iter558 : struct.names)
+          {
+            oprot.writeString(_iter558);
+          }
+        }
+      }
+      if (struct.isSetGet_col_stats()) {
+        oprot.writeBool(struct.get_col_stats);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNamesRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.db_name = iprot.readString();
+      struct.setDb_nameIsSet(true);
+      struct.tbl_name = iprot.readString();
+      struct.setTbl_nameIsSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list559 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.names = new ArrayList<String>(_list559.size);
+          String _elem560;
+          for (int _i561 = 0; _i561 < _list559.size; ++_i561)
+          {
+            _elem560 = iprot.readString();
+            struct.names.add(_elem560);
+          }
+        }
+        struct.setNamesIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.get_col_stats = iprot.readBool();
+        struct.setGet_col_statsIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesResult.java
new file mode 100644
index 0000000..d32e4c2
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsByNamesResult.java
@@ -0,0 +1,443 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetPartitionsByNamesResult implements org.apache.thrift.TBase<GetPartitionsByNamesResult, GetPartitionsByNamesResult._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsByNamesResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsByNamesResult");
+
+  private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetPartitionsByNamesResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsByNamesResultTupleSchemeFactory());
+  }
+
+  private List<Partition> partitions; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PARTITIONS((short)1, "partitions");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PARTITIONS
+          return PARTITIONS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PARTITIONS, new org.apache.thrift.meta_data.FieldMetaData("partitions", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Partition.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsByNamesResult.class, metaDataMap);
+  }
+
+  public GetPartitionsByNamesResult() {
+  }
+
+  public GetPartitionsByNamesResult(
+    List<Partition> partitions)
+  {
+    this();
+    this.partitions = partitions;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsByNamesResult(GetPartitionsByNamesResult other) {
+    if (other.isSetPartitions()) {
+      List<Partition> __this__partitions = new ArrayList<Partition>(other.partitions.size());
+      for (Partition other_element : other.partitions) {
+        __this__partitions.add(new Partition(other_element));
+      }
+      this.partitions = __this__partitions;
+    }
+  }
+
+  public GetPartitionsByNamesResult deepCopy() {
+    return new GetPartitionsByNamesResult(this);
+  }
+
+  @Override
+  public void clear() {
+    this.partitions = null;
+  }
+
+  public int getPartitionsSize() {
+    return (this.partitions == null) ? 0 : this.partitions.size();
+  }
+
+  public java.util.Iterator<Partition> getPartitionsIterator() {
+    return (this.partitions == null) ? null : this.partitions.iterator();
+  }
+
+  public void addToPartitions(Partition elem) {
+    if (this.partitions == null) {
+      this.partitions = new ArrayList<Partition>();
+    }
+    this.partitions.add(elem);
+  }
+
+  public List<Partition> getPartitions() {
+    return this.partitions;
+  }
+
+  public void setPartitions(List<Partition> partitions) {
+    this.partitions = partitions;
+  }
+
+  public void unsetPartitions() {
+    this.partitions = null;
+  }
+
+  /** Returns true if field partitions is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitions() {
+    return this.partitions != null;
+  }
+
+  public void setPartitionsIsSet(boolean value) {
+    if (!value) {
+      this.partitions = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PARTITIONS:
+      if (value == null) {
+        unsetPartitions();
+      } else {
+        setPartitions((List<Partition>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PARTITIONS:
+      return getPartitions();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PARTITIONS:
+      return isSetPartitions();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsByNamesResult)
+      return this.equals((GetPartitionsByNamesResult)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsByNamesResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_partitions = true && this.isSetPartitions();
+    boolean that_present_partitions = true && that.isSetPartitions();
+    if (this_present_partitions || that_present_partitions) {
+      if (!(this_present_partitions && that_present_partitions))
+        return false;
+      if (!this.partitions.equals(that.partitions))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_partitions = true && (isSetPartitions());
+    list.add(present_partitions);
+    if (present_partitions)
+      list.add(partitions);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsByNamesResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetPartitions()).compareTo(other.isSetPartitions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitions, other.partitions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetPartitionsByNamesResult(");
+    boolean first = true;
+
+    sb.append("partitions:");
+    if (this.partitions == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.partitions);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetPartitions()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'partitions' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetPartitionsByNamesResultStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsByNamesResultStandardScheme getScheme() {
+      return new GetPartitionsByNamesResultStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsByNamesResultStandardScheme extends StandardScheme<GetPartitionsByNamesResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsByNamesResult struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PARTITIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list562 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list562.size);
+                Partition _elem563;
+                for (int _i564 = 0; _i564 < _list562.size; ++_i564)
+                {
+                  _elem563 = new Partition();
+                  _elem563.read(iprot);
+                  struct.partitions.add(_elem563);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPartitionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetPartitionsByNamesResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.partitions != null) {
+        oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
+          for (Partition _iter565 : struct.partitions)
+          {
+            _iter565.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsByNamesResultTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsByNamesResultTupleScheme getScheme() {
+      return new GetPartitionsByNamesResultTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsByNamesResultTupleScheme extends TupleScheme<GetPartitionsByNamesResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNamesResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.partitions.size());
+        for (Partition _iter566 : struct.partitions)
+        {
+          _iter566.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsByNamesResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list567.size);
+        Partition _elem568;
+        for (int _i569 = 0; _i569 < _list567.size; ++_i569)
+        {
+          _elem568 = new Partition();
+          _elem568.read(iprot);
+          struct.partitions.add(_elem568);
+        }
+      }
+      struct.setPartitionsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
index b5a2b68..27c81f7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
@@ -444,13 +444,13 @@ import org.slf4j.LoggerFactory;
           case 8: // FILTERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list992 = iprot.readListBegin();
-                struct.filters = new ArrayList<String>(_list992.size);
-                String _elem993;
-                for (int _i994 = 0; _i994 < _list992.size; ++_i994)
+                org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
+                struct.filters = new ArrayList<String>(_list1008.size);
+                String _elem1009;
+                for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
                 {
-                  _elem993 = iprot.readString();
-                  struct.filters.add(_elem993);
+                  _elem1009 = iprot.readString();
+                  struct.filters.add(_elem1009);
                 }
                 iprot.readListEnd();
               }
@@ -484,9 +484,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILTERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filters.size()));
-            for (String _iter995 : struct.filters)
+            for (String _iter1011 : struct.filters)
             {
-              oprot.writeString(_iter995);
+              oprot.writeString(_iter1011);
             }
             oprot.writeListEnd();
           }
@@ -524,9 +524,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilters()) {
         {
           oprot.writeI32(struct.filters.size());
-          for (String _iter996 : struct.filters)
+          for (String _iter1012 : struct.filters)
           {
-            oprot.writeString(_iter996);
+            oprot.writeString(_iter1012);
           }
         }
       }
@@ -542,13 +542,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list997 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filters = new ArrayList<String>(_list997.size);
-          String _elem998;
-          for (int _i999 = 0; _i999 < _list997.size; ++_i999)
+          org.apache.thrift.protocol.TList _list1013 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filters = new ArrayList<String>(_list1013.size);
+          String _elem1014;
+          for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
           {
-            _elem998 = iprot.readString();
-            struct.filters.add(_elem998);
+            _elem1014 = iprot.readString();
+            struct.filters.add(_elem1014);
           }
         }
         struct.setFiltersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
index e6c9c06..30bae47 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
@@ -509,13 +509,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FIELD_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list984 = iprot.readListBegin();
-                struct.fieldList = new ArrayList<String>(_list984.size);
-                String _elem985;
-                for (int _i986 = 0; _i986 < _list984.size; ++_i986)
+                org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin();
+                struct.fieldList = new ArrayList<String>(_list1000.size);
+                String _elem1001;
+                for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
                 {
-                  _elem985 = iprot.readString();
-                  struct.fieldList.add(_elem985);
+                  _elem1001 = iprot.readString();
+                  struct.fieldList.add(_elem1001);
                 }
                 iprot.readListEnd();
               }
@@ -557,9 +557,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FIELD_LIST_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fieldList.size()));
-          for (String _iter987 : struct.fieldList)
+          for (String _iter1003 : struct.fieldList)
           {
-            oprot.writeString(_iter987);
+            oprot.writeString(_iter1003);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +606,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFieldList()) {
         {
           oprot.writeI32(struct.fieldList.size());
-          for (String _iter988 : struct.fieldList)
+          for (String _iter1004 : struct.fieldList)
           {
-            oprot.writeString(_iter988);
+            oprot.writeString(_iter1004);
           }
         }
       }
@@ -626,13 +626,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list989 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.fieldList = new ArrayList<String>(_list989.size);
-          String _elem990;
-          for (int _i991 = 0; _i991 < _list989.size; ++_i991)
+          org.apache.thrift.protocol.TList _list1005 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.fieldList = new ArrayList<String>(_list1005.size);
+          String _elem1006;
+          for (int _i1007 = 0; _i1007 < _list1005.size; ++_i1007)
           {
-            _elem990 = iprot.readString();
-            struct.fieldList.add(_elem990);
+            _elem1006 = iprot.readString();
+            struct.fieldList.add(_elem1006);
           }
         }
         struct.setFieldListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
index 7ec107e..97e24f8 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
@@ -960,13 +960,13 @@ import org.slf4j.LoggerFactory;
           case 6: // GROUP_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
-                struct.groupNames = new ArrayList<String>(_list1008.size);
-                String _elem1009;
-                for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
+                org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin();
+                struct.groupNames = new ArrayList<String>(_list1024.size);
+                String _elem1025;
+                for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
                 {
-                  _elem1009 = iprot.readString();
-                  struct.groupNames.add(_elem1009);
+                  _elem1025 = iprot.readString();
+                  struct.groupNames.add(_elem1025);
                 }
                 iprot.readListEnd();
               }
@@ -1040,9 +1040,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size()));
-            for (String _iter1011 : struct.groupNames)
+            for (String _iter1027 : struct.groupNames)
             {
-              oprot.writeString(_iter1011);
+              oprot.writeString(_iter1027);
             }
             oprot.writeListEnd();
           }
@@ -1120,9 +1120,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetGroupNames()) {
         {
           oprot.writeI32(struct.groupNames.size());
-          for (String _iter1012 : struct.groupNames)
+          for (String _iter1028 : struct.groupNames)
           {
-            oprot.writeString(_iter1012);
+            oprot.writeString(_iter1028);
           }
         }
       }
@@ -1160,13 +1160,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TList _list1013 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.groupNames = new ArrayList<String>(_list1013.size);
-          String _elem1014;
-          for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
+          org.apache.thrift.protocol.TList _list1029 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.groupNames = new ArrayList<String>(_list1029.size);
+          String _elem1030;
+          for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031)
           {
-            _elem1014 = iprot.readString();
-            struct.groupNames.add(_elem1014);
+            _elem1030 = iprot.readString();
+            struct.groupNames.add(_elem1030);
           }
         }
         struct.setGroupNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
index faac848..b8216c8 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // PARTITION_SPEC
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin();
-                struct.partitionSpec = new ArrayList<PartitionSpec>(_list1000.size);
-                PartitionSpec _elem1001;
-                for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
+                org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
+                struct.partitionSpec = new ArrayList<PartitionSpec>(_list1016.size);
+                PartitionSpec _elem1017;
+                for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
                 {
-                  _elem1001 = new PartitionSpec();
-                  _elem1001.read(iprot);
-                  struct.partitionSpec.add(_elem1001);
+                  _elem1017 = new PartitionSpec();
+                  _elem1017.read(iprot);
+                  struct.partitionSpec.add(_elem1017);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITION_SPEC_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionSpec.size()));
-          for (PartitionSpec _iter1003 : struct.partitionSpec)
+          for (PartitionSpec _iter1019 : struct.partitionSpec)
           {
-            _iter1003.write(oprot);
+            _iter1019.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionSpec()) {
         {
           oprot.writeI32(struct.partitionSpec.size());
-          for (PartitionSpec _iter1004 : struct.partitionSpec)
+          for (PartitionSpec _iter1020 : struct.partitionSpec)
           {
-            _iter1004.write(oprot);
+            _iter1020.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list1005 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.partitionSpec = new ArrayList<PartitionSpec>(_list1005.size);
-          PartitionSpec _elem1006;
-          for (int _i1007 = 0; _i1007 < _list1005.size; ++_i1007)
+          org.apache.thrift.protocol.TList _list1021 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionSpec = new ArrayList<PartitionSpec>(_list1021.size);
+          PartitionSpec _elem1022;
+          for (int _i1023 = 0; _i1023 < _list1021.size; ++_i1023)
           {
-            _elem1006 = new PartitionSpec();
-            _elem1006.read(iprot);
-            struct.partitionSpec.add(_elem1006);
+            _elem1022 = new PartitionSpec();
+            _elem1022.read(iprot);
+            struct.partitionSpec.add(_elem1022);
           }
         }
         struct.setPartitionSpecIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index da36157..ed9b709 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -606,13 +606,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list872.size);
-                String _elem873;
-                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
+                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list888.size);
+                String _elem889;
+                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
                 {
-                  _elem873 = iprot.readString();
-                  struct.tblNames.add(_elem873);
+                  _elem889 = iprot.readString();
+                  struct.tblNames.add(_elem889);
                 }
                 iprot.readListEnd();
               }
@@ -661,9 +661,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter875 : struct.tblNames)
+            for (String _iter891 : struct.tblNames)
             {
-              oprot.writeString(_iter875);
+              oprot.writeString(_iter891);
             }
             oprot.writeListEnd();
           }
@@ -716,9 +716,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter876 : struct.tblNames)
+          for (String _iter892 : struct.tblNames)
           {
-            oprot.writeString(_iter876);
+            oprot.writeString(_iter892);
           }
         }
       }
@@ -738,13 +738,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list877.size);
-          String _elem878;
-          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
+          org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list893.size);
+          String _elem894;
+          for (int _i895 = 0; _i895 < _list893.size; ++_i895)
           {
-            _elem878 = iprot.readString();
-            struct.tblNames.add(_elem878);
+            _elem894 = iprot.readString();
+            struct.tblNames.add(_elem894);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index b3cfc88..31584b0 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list880.size);
-                Table _elem881;
-                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
+                org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list896.size);
+                Table _elem897;
+                for (int _i898 = 0; _i898 < _list896.size; ++_i898)
                 {
-                  _elem881 = new Table();
-                  _elem881.read(iprot);
-                  struct.tables.add(_elem881);
+                  _elem897 = new Table();
+                  _elem897.read(iprot);
+                  struct.tables.add(_elem897);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter883 : struct.tables)
+          for (Table _iter899 : struct.tables)
           {
-            _iter883.write(oprot);
+            _iter899.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter884 : struct.tables)
+        for (Table _iter900 : struct.tables)
         {
-          _iter884.write(oprot);
+          _iter900.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list885.size);
-        Table _elem886;
-        for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+        org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list901.size);
+        Table _elem902;
+        for (int _i903 = 0; _i903 < _list901.size; ++_i903)
         {
-          _elem886 = new Table();
-          _elem886.read(iprot);
-          struct.tables.add(_elem886);
+          _elem902 = new Table();
+          _elem902.read(iprot);
+          struct.tables.add(_elem902);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
index 1be6d23..70bfbd2 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -513,13 +513,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FULL_TABLE_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
-                struct.fullTableNames = new ArrayList<String>(_list626.size);
-                String _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                struct.fullTableNames = new ArrayList<String>(_list642.size);
+                String _elem643;
+                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                 {
-                  _elem627 = iprot.readString();
-                  struct.fullTableNames.add(_elem627);
+                  _elem643 = iprot.readString();
+                  struct.fullTableNames.add(_elem643);
                 }
                 iprot.readListEnd();
               }
@@ -561,9 +561,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FULL_TABLE_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fullTableNames.size()));
-          for (String _iter629 : struct.fullTableNames)
+          for (String _iter645 : struct.fullTableNames)
           {
-            oprot.writeString(_iter629);
+            oprot.writeString(_iter645);
           }
           oprot.writeListEnd();
         }
@@ -600,9 +600,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fullTableNames.size());
-        for (String _iter630 : struct.fullTableNames)
+        for (String _iter646 : struct.fullTableNames)
         {
-          oprot.writeString(_iter630);
+          oprot.writeString(_iter646);
         }
       }
       BitSet optionals = new BitSet();
@@ -625,13 +625,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.fullTableNames = new ArrayList<String>(_list631.size);
-        String _elem632;
-        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
+        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.fullTableNames = new ArrayList<String>(_list647.size);
+        String _elem648;
+        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
         {
-          _elem632 = iprot.readString();
-          struct.fullTableNames.add(_elem632);
+          _elem648 = iprot.readString();
+          struct.fullTableNames.add(_elem648);
         }
       }
       struct.setFullTableNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
index 6c3617e..fc94183 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TBL_VALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list642.size);
-                TableValidWriteIds _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list658.size);
+                TableValidWriteIds _elem659;
+                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                 {
-                  _elem643 = new TableValidWriteIds();
-                  _elem643.read(iprot);
-                  struct.tblValidWriteIds.add(_elem643);
+                  _elem659 = new TableValidWriteIds();
+                  _elem659.read(iprot);
+                  struct.tblValidWriteIds.add(_elem659);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TBL_VALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tblValidWriteIds.size()));
-          for (TableValidWriteIds _iter645 : struct.tblValidWriteIds)
+          for (TableValidWriteIds _iter661 : struct.tblValidWriteIds)
           {
-            _iter645.write(oprot);
+            _iter661.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tblValidWriteIds.size());
-        for (TableValidWriteIds _iter646 : struct.tblValidWriteIds)
+        for (TableValidWriteIds _iter662 : struct.tblValidWriteIds)
         {
-          _iter646.write(oprot);
+          _iter662.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list647.size);
-        TableValidWriteIds _elem648;
-        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+        org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list663.size);
+        TableValidWriteIds _elem664;
+        for (int _i665 = 0; _i665 < _list663.size; ++_i665)
         {
-          _elem648 = new TableValidWriteIds();
-          _elem648.read(iprot);
-          struct.tblValidWriteIds.add(_elem648);
+          _elem664 = new TableValidWriteIds();
+          _elem664.read(iprot);
+          struct.tblValidWriteIds.add(_elem664);
         }
       }
       struct.setTblValidWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 761ddb3..cfc8740 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set690 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set690.size);
-                long _elem691;
-                for (int _i692 = 0; _i692 < _set690.size; ++_i692)
+                org.apache.thrift.protocol.TSet _set706 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set706.size);
+                long _elem707;
+                for (int _i708 = 0; _i708 < _set706.size; ++_i708)
                 {
-                  _elem691 = iprot.readI64();
-                  struct.aborted.add(_elem691);
+                  _elem707 = iprot.readI64();
+                  struct.aborted.add(_elem707);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set693 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set693.size);
-                long _elem694;
-                for (int _i695 = 0; _i695 < _set693.size; ++_i695)
+                org.apache.thrift.protocol.TSet _set709 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set709.size);
+                long _elem710;
+                for (int _i711 = 0; _i711 < _set709.size; ++_i711)
                 {
-                  _elem694 = iprot.readI64();
-                  struct.nosuch.add(_elem694);
+                  _elem710 = iprot.readI64();
+                  struct.nosuch.add(_elem710);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter696 : struct.aborted)
+          for (long _iter712 : struct.aborted)
           {
-            oprot.writeI64(_iter696);
+            oprot.writeI64(_iter712);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter697 : struct.nosuch)
+          for (long _iter713 : struct.nosuch)
           {
-            oprot.writeI64(_iter697);
+            oprot.writeI64(_iter713);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter698 : struct.aborted)
+        for (long _iter714 : struct.aborted)
         {
-          oprot.writeI64(_iter698);
+          oprot.writeI64(_iter714);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter699 : struct.nosuch)
+        for (long _iter715 : struct.nosuch)
         {
-          oprot.writeI64(_iter699);
+          oprot.writeI64(_iter715);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set700 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set700.size);
-        long _elem701;
-        for (int _i702 = 0; _i702 < _set700.size; ++_i702)
+        org.apache.thrift.protocol.TSet _set716 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set716.size);
+        long _elem717;
+        for (int _i718 = 0; _i718 < _set716.size; ++_i718)
         {
-          _elem701 = iprot.readI64();
-          struct.aborted.add(_elem701);
+          _elem717 = iprot.readI64();
+          struct.aborted.add(_elem717);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set703 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set703.size);
-        long _elem704;
-        for (int _i705 = 0; _i705 < _set703.size; ++_i705)
+        org.apache.thrift.protocol.TSet _set719 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set719.size);
+        long _elem720;
+        for (int _i721 = 0; _i721 < _set719.size; ++_i721)
         {
-          _elem704 = iprot.readI64();
-          struct.nosuch.add(_elem704);
+          _elem720 = iprot.readI64();
+          struct.nosuch.add(_elem720);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index f1ba643..0279ee1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -636,13 +636,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list756.size);
-                String _elem757;
-                for (int _i758 = 0; _i758 < _list756.size; ++_i758)
+                org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list772.size);
+                String _elem773;
+                for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                 {
-                  _elem757 = iprot.readString();
-                  struct.filesAdded.add(_elem757);
+                  _elem773 = iprot.readString();
+                  struct.filesAdded.add(_elem773);
                 }
                 iprot.readListEnd();
               }
@@ -654,13 +654,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list759 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list759.size);
-                String _elem760;
-                for (int _i761 = 0; _i761 < _list759.size; ++_i761)
+                org.apache.thrift.protocol.TList _list775 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list775.size);
+                String _elem776;
+                for (int _i777 = 0; _i777 < _list775.size; ++_i777)
                 {
-                  _elem760 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem760);
+                  _elem776 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem776);
                 }
                 iprot.readListEnd();
               }
@@ -672,13 +672,13 @@ import org.slf4j.LoggerFactory;
           case 4: // SUB_DIRECTORY_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list762 = iprot.readListBegin();
-                struct.subDirectoryList = new ArrayList<String>(_list762.size);
-                String _elem763;
-                for (int _i764 = 0; _i764 < _list762.size; ++_i764)
+                org.apache.thrift.protocol.TList _list778 = iprot.readListBegin();
+                struct.subDirectoryList = new ArrayList<String>(_list778.size);
+                String _elem779;
+                for (int _i780 = 0; _i780 < _list778.size; ++_i780)
                 {
-                  _elem763 = iprot.readString();
-                  struct.subDirectoryList.add(_elem763);
+                  _elem779 = iprot.readString();
+                  struct.subDirectoryList.add(_elem779);
                 }
                 iprot.readListEnd();
               }
@@ -709,9 +709,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter765 : struct.filesAdded)
+          for (String _iter781 : struct.filesAdded)
           {
-            oprot.writeString(_iter765);
+            oprot.writeString(_iter781);
           }
           oprot.writeListEnd();
         }
@@ -722,9 +722,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter766 : struct.filesAddedChecksum)
+            for (String _iter782 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter766);
+              oprot.writeString(_iter782);
             }
             oprot.writeListEnd();
           }
@@ -736,9 +736,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUB_DIRECTORY_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.subDirectoryList.size()));
-            for (String _iter767 : struct.subDirectoryList)
+            for (String _iter783 : struct.subDirectoryList)
             {
-              oprot.writeString(_iter767);
+              oprot.writeString(_iter783);
             }
             oprot.writeListEnd();
           }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter768 : struct.filesAdded)
+        for (String _iter784 : struct.filesAdded)
         {
-          oprot.writeString(_iter768);
+          oprot.writeString(_iter784);
         }
       }
       BitSet optionals = new BitSet();
@@ -786,18 +786,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter769 : struct.filesAddedChecksum)
+          for (String _iter785 : struct.filesAddedChecksum)
           {
-            oprot.writeString(_iter769);
+            oprot.writeString(_iter785);
           }
         }
       }
       if (struct.isSetSubDirectoryList()) {
         {
           oprot.writeI32(struct.subDirectoryList.size());
-          for (String _iter770 : struct.subDirectoryList)
+          for (String _iter786 : struct.subDirectoryList)
           {
-            oprot.writeString(_iter770);
+            oprot.writeString(_iter786);
           }
         }
       }
@@ -807,13 +807,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list771 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list771.size);
-        String _elem772;
-        for (int _i773 = 0; _i773 < _list771.size; ++_i773)
+        org.apache.thrift.protocol.TList _list787 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list787.size);
+        String _elem788;
+        for (int _i789 = 0; _i789 < _list787.size; ++_i789)
         {
-          _elem772 = iprot.readString();
-          struct.filesAdded.add(_elem772);
+          _elem788 = iprot.readString();
+          struct.filesAdded.add(_elem788);
         }
       }
       struct.setFilesAddedIsSet(true);
@@ -824,26 +824,26 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list774 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list774.size);
-          String _elem775;
-          for (int _i776 = 0; _i776 < _list774.size; ++_i776)
+          org.apache.thrift.protocol.TList _list790 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list790.size);
+          String _elem791;
+          for (int _i792 = 0; _i792 < _list790.size; ++_i792)
           {
-            _elem775 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem775);
+            _elem791 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem791);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.subDirectoryList = new ArrayList<String>(_list777.size);
-          String _elem778;
-          for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+          org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.subDirectoryList = new ArrayList<String>(_list793.size);
+          String _elem794;
+          for (int _i795 = 0; _i795 < _list793.size; ++_i795)
           {
-            _elem778 = iprot.readString();
-            struct.subDirectoryList.add(_elem778);
+            _elem794 = iprot.readString();
+            struct.subDirectoryList.add(_elem794);
           }
         }
         struct.setSubDirectoryListIsSet(true);


[07/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 1bdbbbf..e0431e5 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -208,6 +208,8 @@ import org.slf4j.LoggerFactory;
 
     public List<Partition> get_partitions_by_names(String db_name, String tbl_name, List<String> names) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
 
+    public GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest req) throws MetaException, NoSuchObjectException, org.apache.thrift.TException;
+
     public void alter_partition(String db_name, String tbl_name, Partition new_part) throws InvalidOperationException, MetaException, org.apache.thrift.TException;
 
     public void alter_partitions(String db_name, String tbl_name, List<Partition> new_parts) throws InvalidOperationException, MetaException, org.apache.thrift.TException;
@@ -656,6 +658,8 @@ import org.slf4j.LoggerFactory;
 
     public void get_partitions_by_names(String db_name, String tbl_name, List<String> names, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_partitions_by_names_req(GetPartitionsByNamesRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void alter_partition(String db_name, String tbl_name, Partition new_part, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void alter_partitions(String db_name, String tbl_name, List<Partition> new_parts, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -3460,6 +3464,35 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_partitions_by_names failed: unknown result");
     }
 
+    public GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest req) throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      send_get_partitions_by_names_req(req);
+      return recv_get_partitions_by_names_req();
+    }
+
+    public void send_get_partitions_by_names_req(GetPartitionsByNamesRequest req) throws org.apache.thrift.TException
+    {
+      get_partitions_by_names_req_args args = new get_partitions_by_names_req_args();
+      args.setReq(req);
+      sendBase("get_partitions_by_names_req", args);
+    }
+
+    public GetPartitionsByNamesResult recv_get_partitions_by_names_req() throws MetaException, NoSuchObjectException, org.apache.thrift.TException
+    {
+      get_partitions_by_names_req_result result = new get_partitions_by_names_req_result();
+      receiveBase(result, "get_partitions_by_names_req");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_partitions_by_names_req failed: unknown result");
+    }
+
     public void alter_partition(String db_name, String tbl_name, Partition new_part) throws InvalidOperationException, MetaException, org.apache.thrift.TException
     {
       send_alter_partition(db_name, tbl_name, new_part);
@@ -10309,6 +10342,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void get_partitions_by_names_req(GetPartitionsByNamesRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_partitions_by_names_req_call method_call = new get_partitions_by_names_req_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_by_names_req_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetPartitionsByNamesRequest req;
+      public get_partitions_by_names_req_call(GetPartitionsByNamesRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_partitions_by_names_req", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_partitions_by_names_req_args args = new get_partitions_by_names_req_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetPartitionsByNamesResult getResult() throws MetaException, NoSuchObjectException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_partitions_by_names_req();
+      }
+    }
+
     public void alter_partition(String db_name, String tbl_name, Partition new_part, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       alter_partition_call method_call = new alter_partition_call(db_name, tbl_name, new_part, resultHandler, this, ___protocolFactory, ___transport);
@@ -15006,6 +15071,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_partitions_by_expr", new get_partitions_by_expr());
       processMap.put("get_num_partitions_by_filter", new get_num_partitions_by_filter());
       processMap.put("get_partitions_by_names", new get_partitions_by_names());
+      processMap.put("get_partitions_by_names_req", new get_partitions_by_names_req());
       processMap.put("alter_partition", new alter_partition());
       processMap.put("alter_partitions", new alter_partitions());
       processMap.put("alter_partitions_with_environment_context", new alter_partitions_with_environment_context());
@@ -17349,6 +17415,32 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_by_names_req<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_partitions_by_names_req_args> {
+      public get_partitions_by_names_req() {
+        super("get_partitions_by_names_req");
+      }
+
+      public get_partitions_by_names_req_args getEmptyArgsInstance() {
+        return new get_partitions_by_names_req_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_partitions_by_names_req_result getResult(I iface, get_partitions_by_names_req_args args) throws org.apache.thrift.TException {
+        get_partitions_by_names_req_result result = new get_partitions_by_names_req_result();
+        try {
+          result.success = iface.get_partitions_by_names_req(args.req);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_partition<I extends Iface> extends org.apache.thrift.ProcessFunction<I, alter_partition_args> {
       public alter_partition() {
         super("alter_partition");
@@ -20916,6 +21008,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_partitions_by_expr", new get_partitions_by_expr());
       processMap.put("get_num_partitions_by_filter", new get_num_partitions_by_filter());
       processMap.put("get_partitions_by_names", new get_partitions_by_names());
+      processMap.put("get_partitions_by_names_req", new get_partitions_by_names_req());
       processMap.put("alter_partition", new alter_partition());
       processMap.put("alter_partitions", new alter_partitions());
       processMap.put("alter_partitions_with_environment_context", new alter_partitions_with_environment_context());
@@ -26278,6 +26371,68 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_by_names_req<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_partitions_by_names_req_args, GetPartitionsByNamesResult> {
+      public get_partitions_by_names_req() {
+        super("get_partitions_by_names_req");
+      }
+
+      public get_partitions_by_names_req_args getEmptyArgsInstance() {
+        return new get_partitions_by_names_req_args();
+      }
+
+      public AsyncMethodCallback<GetPartitionsByNamesResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetPartitionsByNamesResult>() { 
+          public void onComplete(GetPartitionsByNamesResult o) {
+            get_partitions_by_names_req_result result = new get_partitions_by_names_req_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_partitions_by_names_req_result result = new get_partitions_by_names_req_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_partitions_by_names_req_args args, org.apache.thrift.async.AsyncMethodCallback<GetPartitionsByNamesResult> resultHandler) throws TException {
+        iface.get_partitions_by_names_req(args.req,resultHandler);
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_partition<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_partition_args, Void> {
       public alter_partition() {
         super("alter_partition");
@@ -44453,13 +44608,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1016.size);
-                  String _elem1017;
-                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
+                  org.apache.thrift.protocol.TList _list1032 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1032.size);
+                  String _elem1033;
+                  for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034)
                   {
-                    _elem1017 = iprot.readString();
-                    struct.success.add(_elem1017);
+                    _elem1033 = iprot.readString();
+                    struct.success.add(_elem1033);
                   }
                   iprot.readListEnd();
                 }
@@ -44494,9 +44649,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1019 : struct.success)
+            for (String _iter1035 : struct.success)
             {
-              oprot.writeString(_iter1019);
+              oprot.writeString(_iter1035);
             }
             oprot.writeListEnd();
           }
@@ -44535,9 +44690,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1020 : struct.success)
+            for (String _iter1036 : struct.success)
             {
-              oprot.writeString(_iter1020);
+              oprot.writeString(_iter1036);
             }
           }
         }
@@ -44552,13 +44707,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1021 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1021.size);
-            String _elem1022;
-            for (int _i1023 = 0; _i1023 < _list1021.size; ++_i1023)
+            org.apache.thrift.protocol.TList _list1037 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1037.size);
+            String _elem1038;
+            for (int _i1039 = 0; _i1039 < _list1037.size; ++_i1039)
             {
-              _elem1022 = iprot.readString();
-              struct.success.add(_elem1022);
+              _elem1038 = iprot.readString();
+              struct.success.add(_elem1038);
             }
           }
           struct.setSuccessIsSet(true);
@@ -45212,13 +45367,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1024.size);
-                  String _elem1025;
-                  for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
+                  org.apache.thrift.protocol.TList _list1040 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1040.size);
+                  String _elem1041;
+                  for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
                   {
-                    _elem1025 = iprot.readString();
-                    struct.success.add(_elem1025);
+                    _elem1041 = iprot.readString();
+                    struct.success.add(_elem1041);
                   }
                   iprot.readListEnd();
                 }
@@ -45253,9 +45408,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1027 : struct.success)
+            for (String _iter1043 : struct.success)
             {
-              oprot.writeString(_iter1027);
+              oprot.writeString(_iter1043);
             }
             oprot.writeListEnd();
           }
@@ -45294,9 +45449,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1028 : struct.success)
+            for (String _iter1044 : struct.success)
             {
-              oprot.writeString(_iter1028);
+              oprot.writeString(_iter1044);
             }
           }
         }
@@ -45311,13 +45466,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1029 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1029.size);
-            String _elem1030;
-            for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031)
+            org.apache.thrift.protocol.TList _list1045 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1045.size);
+            String _elem1046;
+            for (int _i1047 = 0; _i1047 < _list1045.size; ++_i1047)
             {
-              _elem1030 = iprot.readString();
-              struct.success.add(_elem1030);
+              _elem1046 = iprot.readString();
+              struct.success.add(_elem1046);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49924,16 +50079,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1032 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map1032.size);
-                  String _key1033;
-                  Type _val1034;
-                  for (int _i1035 = 0; _i1035 < _map1032.size; ++_i1035)
+                  org.apache.thrift.protocol.TMap _map1048 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map1048.size);
+                  String _key1049;
+                  Type _val1050;
+                  for (int _i1051 = 0; _i1051 < _map1048.size; ++_i1051)
                   {
-                    _key1033 = iprot.readString();
-                    _val1034 = new Type();
-                    _val1034.read(iprot);
-                    struct.success.put(_key1033, _val1034);
+                    _key1049 = iprot.readString();
+                    _val1050 = new Type();
+                    _val1050.read(iprot);
+                    struct.success.put(_key1049, _val1050);
                   }
                   iprot.readMapEnd();
                 }
@@ -49968,10 +50123,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter1036 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1052 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1036.getKey());
-              _iter1036.getValue().write(oprot);
+              oprot.writeString(_iter1052.getKey());
+              _iter1052.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -50010,10 +50165,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter1037 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1053 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1037.getKey());
-              _iter1037.getValue().write(oprot);
+              oprot.writeString(_iter1053.getKey());
+              _iter1053.getValue().write(oprot);
             }
           }
         }
@@ -50028,16 +50183,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1038 = 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*_map1038.size);
-            String _key1039;
-            Type _val1040;
-            for (int _i1041 = 0; _i1041 < _map1038.size; ++_i1041)
+            org.apache.thrift.protocol.TMap _map1054 = 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*_map1054.size);
+            String _key1055;
+            Type _val1056;
+            for (int _i1057 = 0; _i1057 < _map1054.size; ++_i1057)
             {
-              _key1039 = iprot.readString();
-              _val1040 = new Type();
-              _val1040.read(iprot);
-              struct.success.put(_key1039, _val1040);
+              _key1055 = iprot.readString();
+              _val1056 = new Type();
+              _val1056.read(iprot);
+              struct.success.put(_key1055, _val1056);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51072,14 +51227,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1042.size);
-                  FieldSchema _elem1043;
-                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1058.size);
+                  FieldSchema _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1043 = new FieldSchema();
-                    _elem1043.read(iprot);
-                    struct.success.add(_elem1043);
+                    _elem1059 = new FieldSchema();
+                    _elem1059.read(iprot);
+                    struct.success.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -51132,9 +51287,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1045 : struct.success)
+            for (FieldSchema _iter1061 : struct.success)
             {
-              _iter1045.write(oprot);
+              _iter1061.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51189,9 +51344,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1046 : struct.success)
+            for (FieldSchema _iter1062 : struct.success)
             {
-              _iter1046.write(oprot);
+              _iter1062.write(oprot);
             }
           }
         }
@@ -51212,14 +51367,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1047.size);
-            FieldSchema _elem1048;
-            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
+            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1063.size);
+            FieldSchema _elem1064;
+            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
             {
-              _elem1048 = new FieldSchema();
-              _elem1048.read(iprot);
-              struct.success.add(_elem1048);
+              _elem1064 = new FieldSchema();
+              _elem1064.read(iprot);
+              struct.success.add(_elem1064);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52373,14 +52528,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1050.size);
-                  FieldSchema _elem1051;
-                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
+                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1066.size);
+                  FieldSchema _elem1067;
+                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                   {
-                    _elem1051 = new FieldSchema();
-                    _elem1051.read(iprot);
-                    struct.success.add(_elem1051);
+                    _elem1067 = new FieldSchema();
+                    _elem1067.read(iprot);
+                    struct.success.add(_elem1067);
                   }
                   iprot.readListEnd();
                 }
@@ -52433,9 +52588,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1053 : struct.success)
+            for (FieldSchema _iter1069 : struct.success)
             {
-              _iter1053.write(oprot);
+              _iter1069.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52490,9 +52645,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1054 : struct.success)
+            for (FieldSchema _iter1070 : struct.success)
             {
-              _iter1054.write(oprot);
+              _iter1070.write(oprot);
             }
           }
         }
@@ -52513,14 +52668,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1055.size);
-            FieldSchema _elem1056;
-            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
+            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1071.size);
+            FieldSchema _elem1072;
+            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
             {
-              _elem1056 = new FieldSchema();
-              _elem1056.read(iprot);
-              struct.success.add(_elem1056);
+              _elem1072 = new FieldSchema();
+              _elem1072.read(iprot);
+              struct.success.add(_elem1072);
             }
           }
           struct.setSuccessIsSet(true);
@@ -53565,14 +53720,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1058.size);
-                  FieldSchema _elem1059;
-                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
+                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1074.size);
+                  FieldSchema _elem1075;
+                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                   {
-                    _elem1059 = new FieldSchema();
-                    _elem1059.read(iprot);
-                    struct.success.add(_elem1059);
+                    _elem1075 = new FieldSchema();
+                    _elem1075.read(iprot);
+                    struct.success.add(_elem1075);
                   }
                   iprot.readListEnd();
                 }
@@ -53625,9 +53780,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1061 : struct.success)
+            for (FieldSchema _iter1077 : struct.success)
             {
-              _iter1061.write(oprot);
+              _iter1077.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -53682,9 +53837,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1062 : struct.success)
+            for (FieldSchema _iter1078 : struct.success)
             {
-              _iter1062.write(oprot);
+              _iter1078.write(oprot);
             }
           }
         }
@@ -53705,14 +53860,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1063.size);
-            FieldSchema _elem1064;
-            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
+            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1079.size);
+            FieldSchema _elem1080;
+            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
             {
-              _elem1064 = new FieldSchema();
-              _elem1064.read(iprot);
-              struct.success.add(_elem1064);
+              _elem1080 = new FieldSchema();
+              _elem1080.read(iprot);
+              struct.success.add(_elem1080);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54866,14 +55021,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1066.size);
-                  FieldSchema _elem1067;
-                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
+                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1082.size);
+                  FieldSchema _elem1083;
+                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                   {
-                    _elem1067 = new FieldSchema();
-                    _elem1067.read(iprot);
-                    struct.success.add(_elem1067);
+                    _elem1083 = new FieldSchema();
+                    _elem1083.read(iprot);
+                    struct.success.add(_elem1083);
                   }
                   iprot.readListEnd();
                 }
@@ -54926,9 +55081,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1069 : struct.success)
+            for (FieldSchema _iter1085 : struct.success)
             {
-              _iter1069.write(oprot);
+              _iter1085.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54983,9 +55138,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1070 : struct.success)
+            for (FieldSchema _iter1086 : struct.success)
             {
-              _iter1070.write(oprot);
+              _iter1086.write(oprot);
             }
           }
         }
@@ -55006,14 +55161,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1071.size);
-            FieldSchema _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1087.size);
+            FieldSchema _elem1088;
+            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
             {
-              _elem1072 = new FieldSchema();
-              _elem1072.read(iprot);
-              struct.success.add(_elem1072);
+              _elem1088 = new FieldSchema();
+              _elem1088.read(iprot);
+              struct.success.add(_elem1088);
             }
           }
           struct.setSuccessIsSet(true);
@@ -58142,14 +58297,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1074.size);
-                  SQLPrimaryKey _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1090.size);
+                  SQLPrimaryKey _elem1091;
+                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                   {
-                    _elem1075 = new SQLPrimaryKey();
-                    _elem1075.read(iprot);
-                    struct.primaryKeys.add(_elem1075);
+                    _elem1091 = new SQLPrimaryKey();
+                    _elem1091.read(iprot);
+                    struct.primaryKeys.add(_elem1091);
                   }
                   iprot.readListEnd();
                 }
@@ -58161,14 +58316,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1077 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1077.size);
-                  SQLForeignKey _elem1078;
-                  for (int _i1079 = 0; _i1079 < _list1077.size; ++_i1079)
+                  org.apache.thrift.protocol.TList _list1093 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1093.size);
+                  SQLForeignKey _elem1094;
+                  for (int _i1095 = 0; _i1095 < _list1093.size; ++_i1095)
                   {
-                    _elem1078 = new SQLForeignKey();
-                    _elem1078.read(iprot);
-                    struct.foreignKeys.add(_elem1078);
+                    _elem1094 = new SQLForeignKey();
+                    _elem1094.read(iprot);
+                    struct.foreignKeys.add(_elem1094);
                   }
                   iprot.readListEnd();
                 }
@@ -58180,14 +58335,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1080 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1080.size);
-                  SQLUniqueConstraint _elem1081;
-                  for (int _i1082 = 0; _i1082 < _list1080.size; ++_i1082)
+                  org.apache.thrift.protocol.TList _list1096 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1096.size);
+                  SQLUniqueConstraint _elem1097;
+                  for (int _i1098 = 0; _i1098 < _list1096.size; ++_i1098)
                   {
-                    _elem1081 = new SQLUniqueConstraint();
-                    _elem1081.read(iprot);
-                    struct.uniqueConstraints.add(_elem1081);
+                    _elem1097 = new SQLUniqueConstraint();
+                    _elem1097.read(iprot);
+                    struct.uniqueConstraints.add(_elem1097);
                   }
                   iprot.readListEnd();
                 }
@@ -58199,14 +58354,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1083 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1083.size);
-                  SQLNotNullConstraint _elem1084;
-                  for (int _i1085 = 0; _i1085 < _list1083.size; ++_i1085)
+                  org.apache.thrift.protocol.TList _list1099 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1099.size);
+                  SQLNotNullConstraint _elem1100;
+                  for (int _i1101 = 0; _i1101 < _list1099.size; ++_i1101)
                   {
-                    _elem1084 = new SQLNotNullConstraint();
-                    _elem1084.read(iprot);
-                    struct.notNullConstraints.add(_elem1084);
+                    _elem1100 = new SQLNotNullConstraint();
+                    _elem1100.read(iprot);
+                    struct.notNullConstraints.add(_elem1100);
                   }
                   iprot.readListEnd();
                 }
@@ -58218,14 +58373,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1086 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1086.size);
-                  SQLDefaultConstraint _elem1087;
-                  for (int _i1088 = 0; _i1088 < _list1086.size; ++_i1088)
+                  org.apache.thrift.protocol.TList _list1102 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1102.size);
+                  SQLDefaultConstraint _elem1103;
+                  for (int _i1104 = 0; _i1104 < _list1102.size; ++_i1104)
                   {
-                    _elem1087 = new SQLDefaultConstraint();
-                    _elem1087.read(iprot);
-                    struct.defaultConstraints.add(_elem1087);
+                    _elem1103 = new SQLDefaultConstraint();
+                    _elem1103.read(iprot);
+                    struct.defaultConstraints.add(_elem1103);
                   }
                   iprot.readListEnd();
                 }
@@ -58237,14 +58392,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1089 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1089.size);
-                  SQLCheckConstraint _elem1090;
-                  for (int _i1091 = 0; _i1091 < _list1089.size; ++_i1091)
+                  org.apache.thrift.protocol.TList _list1105 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1105.size);
+                  SQLCheckConstraint _elem1106;
+                  for (int _i1107 = 0; _i1107 < _list1105.size; ++_i1107)
                   {
-                    _elem1090 = new SQLCheckConstraint();
-                    _elem1090.read(iprot);
-                    struct.checkConstraints.add(_elem1090);
+                    _elem1106 = new SQLCheckConstraint();
+                    _elem1106.read(iprot);
+                    struct.checkConstraints.add(_elem1106);
                   }
                   iprot.readListEnd();
                 }
@@ -58275,9 +58430,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1092 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1108 : struct.primaryKeys)
             {
-              _iter1092.write(oprot);
+              _iter1108.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58287,9 +58442,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1093 : struct.foreignKeys)
+            for (SQLForeignKey _iter1109 : struct.foreignKeys)
             {
-              _iter1093.write(oprot);
+              _iter1109.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58299,9 +58454,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1094 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1110 : struct.uniqueConstraints)
             {
-              _iter1094.write(oprot);
+              _iter1110.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58311,9 +58466,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1095 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1111 : struct.notNullConstraints)
             {
-              _iter1095.write(oprot);
+              _iter1111.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58323,9 +58478,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1096 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1112 : struct.defaultConstraints)
             {
-              _iter1096.write(oprot);
+              _iter1112.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58335,9 +58490,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1097 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1113 : struct.checkConstraints)
             {
-              _iter1097.write(oprot);
+              _iter1113.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -58389,54 +58544,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1098 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1114 : struct.primaryKeys)
             {
-              _iter1098.write(oprot);
+              _iter1114.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1099 : struct.foreignKeys)
+            for (SQLForeignKey _iter1115 : struct.foreignKeys)
             {
-              _iter1099.write(oprot);
+              _iter1115.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1100 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1116 : struct.uniqueConstraints)
             {
-              _iter1100.write(oprot);
+              _iter1116.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1101 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1117 : struct.notNullConstraints)
             {
-              _iter1101.write(oprot);
+              _iter1117.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1102 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1118 : struct.defaultConstraints)
             {
-              _iter1102.write(oprot);
+              _iter1118.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1103 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1119 : struct.checkConstraints)
             {
-              _iter1103.write(oprot);
+              _iter1119.write(oprot);
             }
           }
         }
@@ -58453,84 +58608,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1104 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1104.size);
-            SQLPrimaryKey _elem1105;
-            for (int _i1106 = 0; _i1106 < _list1104.size; ++_i1106)
+            org.apache.thrift.protocol.TList _list1120 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1120.size);
+            SQLPrimaryKey _elem1121;
+            for (int _i1122 = 0; _i1122 < _list1120.size; ++_i1122)
             {
-              _elem1105 = new SQLPrimaryKey();
-              _elem1105.read(iprot);
-              struct.primaryKeys.add(_elem1105);
+              _elem1121 = new SQLPrimaryKey();
+              _elem1121.read(iprot);
+              struct.primaryKeys.add(_elem1121);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1107.size);
-            SQLForeignKey _elem1108;
-            for (int _i1109 = 0; _i1109 < _list1107.size; ++_i1109)
+            org.apache.thrift.protocol.TList _list1123 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1123.size);
+            SQLForeignKey _elem1124;
+            for (int _i1125 = 0; _i1125 < _list1123.size; ++_i1125)
             {
-              _elem1108 = new SQLForeignKey();
-              _elem1108.read(iprot);
-              struct.foreignKeys.add(_elem1108);
+              _elem1124 = new SQLForeignKey();
+              _elem1124.read(iprot);
+              struct.foreignKeys.add(_elem1124);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1110 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1110.size);
-            SQLUniqueConstraint _elem1111;
-            for (int _i1112 = 0; _i1112 < _list1110.size; ++_i1112)
+            org.apache.thrift.protocol.TList _list1126 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1126.size);
+            SQLUniqueConstraint _elem1127;
+            for (int _i1128 = 0; _i1128 < _list1126.size; ++_i1128)
             {
-              _elem1111 = new SQLUniqueConstraint();
-              _elem1111.read(iprot);
-              struct.uniqueConstraints.add(_elem1111);
+              _elem1127 = new SQLUniqueConstraint();
+              _elem1127.read(iprot);
+              struct.uniqueConstraints.add(_elem1127);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1113 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1113.size);
-            SQLNotNullConstraint _elem1114;
-            for (int _i1115 = 0; _i1115 < _list1113.size; ++_i1115)
+            org.apache.thrift.protocol.TList _list1129 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1129.size);
+            SQLNotNullConstraint _elem1130;
+            for (int _i1131 = 0; _i1131 < _list1129.size; ++_i1131)
             {
-              _elem1114 = new SQLNotNullConstraint();
-              _elem1114.read(iprot);
-              struct.notNullConstraints.add(_elem1114);
+              _elem1130 = new SQLNotNullConstraint();
+              _elem1130.read(iprot);
+              struct.notNullConstraints.add(_elem1130);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1116 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1116.size);
-            SQLDefaultConstraint _elem1117;
-            for (int _i1118 = 0; _i1118 < _list1116.size; ++_i1118)
+            org.apache.thrift.protocol.TList _list1132 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1132.size);
+            SQLDefaultConstraint _elem1133;
+            for (int _i1134 = 0; _i1134 < _list1132.size; ++_i1134)
             {
-              _elem1117 = new SQLDefaultConstraint();
-              _elem1117.read(iprot);
-              struct.defaultConstraints.add(_elem1117);
+              _elem1133 = new SQLDefaultConstraint();
+              _elem1133.read(iprot);
+              struct.defaultConstraints.add(_elem1133);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1119.size);
-            SQLCheckConstraint _elem1120;
-            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
+            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1135.size);
+            SQLCheckConstraint _elem1136;
+            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
             {
-              _elem1120 = new SQLCheckConstraint();
-              _elem1120.read(iprot);
-              struct.checkConstraints.add(_elem1120);
+              _elem1136 = new SQLCheckConstraint();
+              _elem1136.read(iprot);
+              struct.checkConstraints.add(_elem1136);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -67680,13 +67835,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1122.size);
-                  String _elem1123;
-                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
+                  org.apache.thrift.protocol.TList _list1138 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1138.size);
+                  String _elem1139;
+                  for (int _i1140 = 0; _i1140 < _list1138.size; ++_i1140)
                   {
-                    _elem1123 = iprot.readString();
-                    struct.partNames.add(_elem1123);
+                    _elem1139 = iprot.readString();
+                    struct.partNames.add(_elem1139);
                   }
                   iprot.readListEnd();
                 }
@@ -67722,9 +67877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter1125 : struct.partNames)
+            for (String _iter1141 : struct.partNames)
             {
-              oprot.writeString(_iter1125);
+              oprot.writeString(_iter1141);
             }
             oprot.writeListEnd();
           }
@@ -67767,9 +67922,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1126 : struct.partNames)
+            for (String _iter1142 : struct.partNames)
             {
-              oprot.writeString(_iter1126);
+              oprot.writeString(_iter1142);
             }
           }
         }
@@ -67789,13 +67944,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1127.size);
-            String _elem1128;
-            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
+            org.apache.thrift.protocol.TList _list1143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1143.size);
+            String _elem1144;
+            for (int _i1145 = 0; _i1145 < _list1143.size; ++_i1145)
             {
-              _elem1128 = iprot.readString();
-              struct.partNames.add(_elem1128);
+              _elem1144 = iprot.readString();
+              struct.partNames.add(_elem1144);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -69852,13 +70007,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1130 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1130.size);
-                  String _elem1131;
-                  for (int _i1132 = 0; _i1132 < _list1130.size; ++_i1132)
+                  org.apache.thrift.protocol.TList _list1146 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1146.size);
+                  String _elem1147;
+                  for (int _i1148 = 0; _i1148 < _list1146.size; ++_i1148)
                   {
-                    _elem1131 = iprot.readString();
-                    struct.success.add(_elem1131);
+                    _elem1147 = iprot.readString();
+                    struct.success.add(_elem1147);
                   }
                   iprot.readListEnd();
                 }
@@ -69893,9 +70048,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1133 : struct.success)
+            for (String _iter1149 : struct.success)
             {
-              oprot.writeString(_iter1133);
+              oprot.writeString(_iter1149);
             }
             oprot.writeListEnd();
           }
@@ -69934,9 +70089,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1134 : struct.success)
+            for (String _iter1150 : struct.success)
             {
-              oprot.writeString(_iter1134);
+              oprot.writeString(_iter1150);
             }
           }
         }
@@ -69951,13 +70106,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1135.size);
-            String _elem1136;
-            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
+            org.apache.thrift.protocol.TList _list1151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1151.size);
+            String _elem1152;
+            for (int _i1153 = 0; _i1153 < _list1151.size; ++_i1153)
             {
-              _elem1136 = iprot.readString();
-              struct.success.add(_elem1136);
+              _elem1152 = iprot.readString();
+              struct.success.add(_elem1152);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70931,13 +71086,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1138 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1138.size);
-                  String _elem1139;
-                  for (int _i1140 = 0; _i1140 < _list1138.size; ++_i1140)
+                  org.apache.thrift.protocol.TList _list1154 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1154.size);
+                  String _elem1155;
+                  for (int _i1156 = 0; _i1156 < _list1154.size; ++_i1156)
                   {
-                    _elem1139 = iprot.readString();
-                    struct.success.add(_elem1139);
+                    _elem1155 = iprot.readString();
+                    struct.success.add(_elem1155);
                   }
                   iprot.readListEnd();
                 }
@@ -70972,9 +71127,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1141 : struct.success)
+            for (String _iter1157 : struct.success)
             {
-              oprot.writeString(_iter1141);
+              oprot.writeString(_iter1157);
             }
             oprot.writeListEnd();
           }
@@ -71013,9 +71168,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1142 : struct.success)
+            for (String _iter1158 : struct.success)
             {
-              oprot.writeString(_iter1142);
+              oprot.writeString(_iter1158);
             }
           }
         }
@@ -71030,13 +71185,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1143.size);
-            String _elem1144;
-            for (int _i1145 = 0; _i1145 < _list1143.size; ++_i1145)
+            org.apache.thrift.protocol.TList _list1159 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1159.size);
+            String _elem1160;
+            for (int _i1161 = 0; _i1161 < _list1159.size; ++_i1161)
             {
-              _elem1144 = iprot.readString();
-              struct.success.add(_elem1144);
+              _elem1160 = iprot.readString();
+              struct.success.add(_elem1160);
             }
           }
           struct.setSuccessIsSet(true);
@@ -71802,13 +71957,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1146 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1146.size);
-                  String _elem1147;
-                  for (int _i1148 = 0; _i1148 < _list1146.size; ++_i1148)
+                  org.apache.thrift.protocol.TList _list1162 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1162.size);
+                  String _elem1163;
+                  for (int _i1164 = 0; _i1164 < _list1162.size; ++_i1164)
                   {
-                    _elem1147 = iprot.readString();
-                    struct.success.add(_elem1147);
+                    _elem1163 = iprot.readString();
+                    struct.success.add(_elem1163);
                   }
                   iprot.readListEnd();
                 }
@@ -71843,9 +71998,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1149 : struct.success)
+            for (String _iter1165 : struct.success)
             {
-              oprot.writeString(_iter1149);
+              oprot.writeString(_iter1165);
             }
             oprot.writeListEnd();
           }
@@ -71884,9 +72039,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1150 : struct.success)
+            for (String _iter1166 : struct.success)
             {
-              oprot.writeString(_iter1150);
+              oprot.writeString(_iter1166);
             }
           }
         }
@@ -71901,13 +72056,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1151.size);
-            String _elem1152;
-            for (int _i1153 = 0; _i1153 < _list1151.size; ++_i1153)
+            org.apache.thrift.protocol.TList _list1167 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1167.size);
+            String _elem1168;
+            for (int _i1169 = 0; _i1169 < _list1167.size; ++_i1169)
             {
-              _elem1152 = iprot.readString();
-              struct.success.add(_elem1152);
+              _elem1168 = iprot.readString();
+              struct.success.add(_elem1168);
             }
           }
           struct.setSuccessIsSet(true);
@@ -72412,13 +72567,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1154 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1154.size);
-                  String _elem1155;
-                  for (int _i1156 = 0; _i1156 < _list1154.size; ++_i1156)
+                  org.apache.thrift.protocol.TList _list1170 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1170.size);
+                  String _elem1171;
+                  for (int _i1172 = 0; _i1172 < _list1170.size; ++_i1172)
                   {
-                    _elem1155 = iprot.readString();
-                    struct.tbl_types.add(_elem1155);
+                    _elem1171 = iprot.readString();
+                    struct.tbl_types.add(_elem1171);
                   }
                   iprot.readListEnd();
                 }
@@ -72454,9 +72609,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter1157 : struct.tbl_types)
+            for (String _iter1173 : struct.tbl_types)
             {
-              oprot.writeString(_iter1157);
+              oprot.writeString(_iter1173);
             }
             oprot.writeListEnd();
           }
@@ -72499,9 +72654,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1158 : struct.tbl_types)
+            for (String _iter1174 : struct.tbl_types)
             {
-              oprot.writeString(_iter1158);
+              oprot.writeString(_iter1174);
             }
           }
         }
@@ -72521,13 +72676,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1159 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1159.size);
-            String _elem1160;
-            for (int _i1161 = 0; _i1161 < _list1159.size; ++_i1161)
+            org.apache.thrift.protocol.TList _list1175 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1175.size);
+            String _elem1176;
+            for (int _i1177 = 0; _i1177 < _list1175.size; ++_i1177)
             {
-              _elem1160 = iprot.readString();
-              struct.tbl_types.add(_elem1160);
+              _elem1176 = iprot.readString();
+              struct.tbl_types.add(_elem1176);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -72933,14 +73088,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1162 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1162.size);
-                  TableMeta _elem1163;
-                  for (int _i1164 = 0; _i1164 < _list1162.size; ++_i1164)
+                  org.apache.thrift.protocol.TList _list1178 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1178.size);
+                  TableMeta _elem1179;
+                  for (int _i1180 = 0; _i1180 < _list1178.size; ++_i1180)
                   {
-                    _elem1163 = new TableMeta();
-                    _elem1163.read(iprot);
-                    struct.success.add(_elem1163);
+                    _elem1179 = new TableMeta();
+                    _elem1179.read(iprot);
+                    struct.success.add(_elem1179);
                   }
                   iprot.readListEnd();
                 }
@@ -72975,9 +73130,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1165 : struct.success)
+            for (TableMeta _iter1181 : struct.success)
             {
-              _iter1165.write(oprot);
+              _iter1181.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -73016,9 +73171,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1166 : struct.success)
+            for (TableMeta _iter1182 : struct.success)
             {
-              _iter1166.write(oprot);
+              _iter1182.write(oprot);
             }
           }
         }
@@ -73033,14 +73188,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1167 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1167.size);
-            TableMeta _elem1168;
-            for (int _i1169 = 0; _i1169 < _list1167.size; ++_i1169)
+            org.apache.thrift.protocol.TList _list1183 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1183.size);
+            TableMeta _elem1184;
+            for (int _i1185 = 0; _i1185 < _list1183.size; ++_i1185)
             {
-              _elem1168 = new TableMeta();
-              _elem1168.read(iprot);
-              struct.success.add(_elem1168);
+              _elem1184 = new TableMeta();
+              _elem1184.read(iprot);
+              struct.success.add(_elem1184);
             }
           }
           struct.setSuccessIsSet(true);
@@ -73806,13 +73961,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1170 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1170.size);
-                  String _elem1171;
-                  for (int _i1172 = 0; _i1172 < _list1170.size; ++_i1172)
+                  org.apache.thrift.protocol.TList _list1186 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1186.size);
+                  String _elem1187;
+                  for (int _i1188 = 0; _i1188 < _list1186.size; ++_i1188)
                   {
-                    _elem1171 = iprot.readString();
-                    struct.success.add(_elem1171);
+                    _elem1187 = iprot.readString();
+                    struct.success.add(_elem1187);
                   }
                   iprot.readListEnd();
                 }
@@ -73847,9 +74002,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1173 : struct.success)
+            for (String _iter1189 : struct.success)
             {
-              oprot.writeString(_iter1173);
+              oprot.writeString(_iter1189);
             }
             oprot.writeListEnd();
           }
@@ -73888,9 +74043,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1174 : struct.success)
+            for (String _iter1190 : struct.success)
             {
-              oprot.writeString(_iter1174);
+              oprot.writeString(_iter1190);
             }
           }
         }
@@ -73905,13 +74060,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1175 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1175.size);
-            String _elem1176;
-            for (int _i1177 = 0; _i1177 < _list1175.size; ++_i1177)
+            org.apache.thrift.protocol.TList _list1191 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1191.size);
+            String _elem1192;
+            for (int _i1193 = 0; _i1193 < _list1191.size; ++_i1193)
             {
-              _elem1176 = iprot.readString();
-              struct.success.add(_elem1176);
+              _elem1192 = iprot.readString();
+              struct.success.add(_elem1192);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75364,13 +75519,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1178 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1178.size);
-                  String _elem1179;
-                  for (int _i1180 = 0; _i1180 < _list1178.size; ++_i1180)
+                  org.apache.thrift.protocol.TList _list1194 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1194.size);
+                  String _elem1195;
+                  for (int _i1196 = 0; _i1196 < _list1194.size; ++_i1196)
                   {
-                    _elem1179 = iprot.readString();
-                    struct.tbl_names.add(_elem1179);
+                    _elem1195 = iprot.readString();
+                    struct.tbl_names.add(_elem1195);
                   }
                   iprot.readListEnd();
                 }
@@ -75401,9 +75556,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1181 : struct.tbl_names)
+            for (String _iter1197 : struct.tbl_names)
             {
-              oprot.writeString(_iter1181);
+              oprot.writeString(_iter1197);
             }
             oprot.writeListEnd();
           }
@@ -75440,9 +75595,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1182 : struct.tbl_names)
+            for (String _iter1198 : struct.tbl_names)
             {
-              oprot.writeString(_iter1182);
+              oprot.writeString(_iter1198);
             }
           }
         }
@@ -75458,13 +75613,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1183 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1183.size);
-            String _elem1184;
-            for (int _i1185 = 0; _i1185 < _list1183.size; ++_i1185)
+            org.apache.thrift.protocol.TList _list1199 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1199.size);
+            String _elem1200;
+            for (int _i1201 = 0; _i1201 < _list1199.size; ++_i1201)
             {
-              _elem1184 = iprot.readString();
-              struct.tbl_names.add(_elem1184);
+              _elem1200 = iprot.readString();
+              struct.tbl_names.add(_elem1200);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -75789,14 +75944,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1186 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1186.size);
-                  Table _elem1187;
-                  for (int _i1188 = 0; _i1188 < _list1186.size; ++_i1188)
+                  org.apache.thrift.protocol.TList _list1202 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1202.size);
+                  Table _elem1203;
+                  for (int _i1204 = 0; _i1204 < _list1202.size; ++_i1204)
                   {
-                    _elem1187 = new Table();
-                    _elem1187.read(iprot);
-                    struct.success.add(_elem1187);
+                    _elem1203 = new Table();
+                    _elem1203.read(iprot);
+                    struct.success.add(_elem1203);
                   }
                   iprot.readListEnd();
                 }
@@ -75822,9 +75977,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1189 : struct.success)
+            for (Table _iter1205 : struct.success)
             {
-              _iter1189.write(oprot);
+              _iter1205.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -75855,9 +76010,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1190 : struct.success)
+            for (Table _iter1206 : struct.success)
             {
-              _iter1190.write(oprot);
+              _iter1206.write(oprot);
             }
           }
         }
@@ -75869,14 +76024,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1191 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1191.size);
-            Table _elem1192;
-            for (int _i1193 = 0; _i1193 < _list1191.size; ++_i1193)
+            org.apache.thrift.protocol.TList _list1207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1207.size);
+            Table _elem1208;
+            for (int _i1209 = 0; _i1209 < _list1207.size; ++_i1209)
             {
-              _elem1192 = new Table();
-              _elem1192.read(iprot);
-              struct.success.add(_elem1192);
+              _elem1208 = new Table();
+              _elem1208.read(iprot);
+              struct.success.add(_elem1208);
             }
           }
           struct.setSuccessIsSet(true);
@@ -81384,13 +81539,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1194 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1194.size);
-                  String _elem1195;
-                  for (int _i1196 = 0; _i1196 < _list1194.size; ++_i1196)
+                  org.apache.thrift.protocol.TList _list1210 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1210.size);
+                  String _elem1211;
+                  for (int _i1212 = 0; _i1212 < _list1210.size; ++_i1212)
                   {
-                    _elem1195 = iprot.readString();
-                    struct.success.add(_elem1195);
+                    _elem1211 = iprot.readString();
+                    struct.success.add(_elem1211);
                   }
                   iprot.readListEnd();
                 }
@@ -81443,9 +81598,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1197 : struct.success)
+            for (String _iter1213 : struct.success)
             {
-              oprot.writeString(_iter1197);
+              oprot.writeString(_iter1213);
             }
             oprot.writeListEnd();
           }
@@ -81500,9 +81655,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1198 : struct.success)
+            for (String _iter1214 : struct.success)
             {
-              oprot.writeString(_iter1198);
+              oprot.writeString(_iter1214);
             }
           }
         }
@@ -81523,13 +81678,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1199 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1199.size);
-            String _elem1200;
-            for (int _i1201 = 0; _i1201 < _list1199.size; ++_i1201)
+            org.apache.thrift.protocol.TList _list1215 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1215.size);
+            String _elem1216;
+            for (int _i1217 = 0; _i1217 < _list1215.size; ++_i1217)
             {
-              _elem1200 = iprot.readString();
-              struct.success.add(_elem1200);
+              _elem1216 = iprot.readString();
+              struct.success.add(_elem1216);
             }
           }
           struct.setSuccessIsSet(true);
@@ -88326,14 +88481,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1202 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1202.size);
-                  Partition _elem1203;
-                  for (int _i1204 = 0; _i1204 < _list1202.size; ++_i1204)
+                  org.apache.thrift.protocol.TList _list1218 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1218.size);
+                  Partition _elem1219;
+                  for (int _i1220 = 0; _i1220 < _list1218.size; ++_i1220)
                   {
-                    _elem1203 = new Partition();
-                    _elem1203.read(iprot);
-                    struct.new_parts.add(_elem1203);
+                    _elem1219 = new Partition();
+                    _elem1219.read(iprot);
+                    struct.new_parts.add(_elem1219);
                   }
                   iprot.readListEnd();
                 }
@@ -88359,9 +88514,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter1205 : struct.new_parts)
+            for (Partition _iter1221 : struct.new_parts)
             {
-              _iter1205.write(oprot);
+              _iter1221.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -88392,9 +88547,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1206 : struct.new_parts)
+            for (Partition _iter1222 : struct.new_parts)
             {
-              _iter1206.write(oprot);
+              _iter1222.write(oprot);
             }
           }
         }
@@ -88406,14 +88561,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1207 = new org.ap

<TRUNCATED>

[03/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index ba06c54..9aeae9f 100755
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -107,6 +107,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  PartitionsByExprResult get_partitions_by_expr(PartitionsByExprRequest req)')
   print('  i32 get_num_partitions_by_filter(string db_name, string tbl_name, string filter)')
   print('   get_partitions_by_names(string db_name, string tbl_name,  names)')
+  print('  GetPartitionsByNamesResult get_partitions_by_names_req(GetPartitionsByNamesRequest req)')
   print('  void alter_partition(string db_name, string tbl_name, Partition new_part)')
   print('  void alter_partitions(string db_name, string tbl_name,  new_parts)')
   print('  void alter_partitions_with_environment_context(string db_name, string tbl_name,  new_parts, EnvironmentContext environment_context)')
@@ -813,6 +814,12 @@ elif cmd == 'get_partitions_by_names':
     sys.exit(1)
   pp.pprint(client.get_partitions_by_names(args[0],args[1],eval(args[2]),))
 
+elif cmd == 'get_partitions_by_names_req':
+  if len(args) != 1:
+    print('get_partitions_by_names_req requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_partitions_by_names_req(eval(args[0]),))
+
 elif cmd == 'alter_partition':
   if len(args) != 3:
     print('alter_partition requires 3 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 37db81f..eadf300 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -713,6 +713,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_partitions_by_names_req(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
   def alter_partition(self, db_name, tbl_name, new_part):
     """
     Parameters:
@@ -4835,6 +4842,41 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o2
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_partitions_by_names failed: unknown result")
 
+  def get_partitions_by_names_req(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_get_partitions_by_names_req(req)
+    return self.recv_get_partitions_by_names_req()
+
+  def send_get_partitions_by_names_req(self, req):
+    self._oprot.writeMessageBegin('get_partitions_by_names_req', TMessageType.CALL, self._seqid)
+    args = get_partitions_by_names_req_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_partitions_by_names_req(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_partitions_by_names_req_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_partitions_by_names_req failed: unknown result")
+
   def alter_partition(self, db_name, tbl_name, new_part):
     """
     Parameters:
@@ -9694,6 +9736,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_partitions_by_expr"] = Processor.process_get_partitions_by_expr
     self._processMap["get_num_partitions_by_filter"] = Processor.process_get_num_partitions_by_filter
     self._processMap["get_partitions_by_names"] = Processor.process_get_partitions_by_names
+    self._processMap["get_partitions_by_names_req"] = Processor.process_get_partitions_by_names_req
     self._processMap["alter_partition"] = Processor.process_alter_partition
     self._processMap["alter_partitions"] = Processor.process_alter_partitions
     self._processMap["alter_partitions_with_environment_context"] = Processor.process_alter_partitions_with_environment_context
@@ -11978,6 +12021,31 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_partitions_by_names_req(self, seqid, iprot, oprot):
+    args = get_partitions_by_names_req_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_partitions_by_names_req_result()
+    try:
+      result.success = self._handler.get_partitions_by_names_req(args.req)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_partitions_by_names_req", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_alter_partition(self, seqid, iprot, oprot):
     args = alter_partition_args()
     args.read(iprot)
@@ -16953,10 +17021,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype896, _size893) = iprot.readListBegin()
-          for _i897 in xrange(_size893):
-            _elem898 = iprot.readString()
-            self.success.append(_elem898)
+          (_etype910, _size907) = iprot.readListBegin()
+          for _i911 in xrange(_size907):
+            _elem912 = iprot.readString()
+            self.success.append(_elem912)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16979,8 +17047,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 iter899 in self.success:
-        oprot.writeString(iter899)
+      for iter913 in self.success:
+        oprot.writeString(iter913)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17085,10 +17153,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype903, _size900) = iprot.readListBegin()
-          for _i904 in xrange(_size900):
-            _elem905 = iprot.readString()
-            self.success.append(_elem905)
+          (_etype917, _size914) = iprot.readListBegin()
+          for _i918 in xrange(_size914):
+            _elem919 = iprot.readString()
+            self.success.append(_elem919)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17111,8 +17179,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 iter906 in self.success:
-        oprot.writeString(iter906)
+      for iter920 in self.success:
+        oprot.writeString(iter920)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17882,12 +17950,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype908, _vtype909, _size907 ) = iprot.readMapBegin()
-          for _i911 in xrange(_size907):
-            _key912 = iprot.readString()
-            _val913 = Type()
-            _val913.read(iprot)
-            self.success[_key912] = _val913
+          (_ktype922, _vtype923, _size921 ) = iprot.readMapBegin()
+          for _i925 in xrange(_size921):
+            _key926 = iprot.readString()
+            _val927 = Type()
+            _val927.read(iprot)
+            self.success[_key926] = _val927
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17910,9 +17978,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 kiter914,viter915 in self.success.items():
-        oprot.writeString(kiter914)
-        viter915.write(oprot)
+      for kiter928,viter929 in self.success.items():
+        oprot.writeString(kiter928)
+        viter929.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18055,11 +18123,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype919, _size916) = iprot.readListBegin()
-          for _i920 in xrange(_size916):
-            _elem921 = FieldSchema()
-            _elem921.read(iprot)
-            self.success.append(_elem921)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = FieldSchema()
+            _elem935.read(iprot)
+            self.success.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18094,8 +18162,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 iter922 in self.success:
-        iter922.write(oprot)
+      for iter936 in self.success:
+        iter936.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18262,11 +18330,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype926, _size923) = iprot.readListBegin()
-          for _i927 in xrange(_size923):
-            _elem928 = FieldSchema()
-            _elem928.read(iprot)
-            self.success.append(_elem928)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = FieldSchema()
+            _elem942.read(iprot)
+            self.success.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18301,8 +18369,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 iter929 in self.success:
-        iter929.write(oprot)
+      for iter943 in self.success:
+        iter943.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18455,11 +18523,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = FieldSchema()
-            _elem935.read(iprot)
-            self.success.append(_elem935)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = FieldSchema()
+            _elem949.read(iprot)
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18494,8 +18562,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 iter936 in self.success:
-        iter936.write(oprot)
+      for iter950 in self.success:
+        iter950.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18662,11 +18730,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = FieldSchema()
-            _elem942.read(iprot)
-            self.success.append(_elem942)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = FieldSchema()
+            _elem956.read(iprot)
+            self.success.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18701,8 +18769,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 iter943 in self.success:
-        iter943.write(oprot)
+      for iter957 in self.success:
+        iter957.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19155,66 +19223,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = SQLPrimaryKey()
-            _elem949.read(iprot)
-            self.primaryKeys.append(_elem949)
+          (_etype961, _size958) = iprot.readListBegin()
+          for _i962 in xrange(_size958):
+            _elem963 = SQLPrimaryKey()
+            _elem963.read(iprot)
+            self.primaryKeys.append(_elem963)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype953, _size950) = iprot.readListBegin()
-          for _i954 in xrange(_size950):
-            _elem955 = SQLForeignKey()
-            _elem955.read(iprot)
-            self.foreignKeys.append(_elem955)
+          (_etype967, _size964) = iprot.readListBegin()
+          for _i968 in xrange(_size964):
+            _elem969 = SQLForeignKey()
+            _elem969.read(iprot)
+            self.foreignKeys.append(_elem969)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype959, _size956) = iprot.readListBegin()
-          for _i960 in xrange(_size956):
-            _elem961 = SQLUniqueConstraint()
-            _elem961.read(iprot)
-            self.uniqueConstraints.append(_elem961)
+          (_etype973, _size970) = iprot.readListBegin()
+          for _i974 in xrange(_size970):
+            _elem975 = SQLUniqueConstraint()
+            _elem975.read(iprot)
+            self.uniqueConstraints.append(_elem975)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype965, _size962) = iprot.readListBegin()
-          for _i966 in xrange(_size962):
-            _elem967 = SQLNotNullConstraint()
-            _elem967.read(iprot)
-            self.notNullConstraints.append(_elem967)
+          (_etype979, _size976) = iprot.readListBegin()
+          for _i980 in xrange(_size976):
+            _elem981 = SQLNotNullConstraint()
+            _elem981.read(iprot)
+            self.notNullConstraints.append(_elem981)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype971, _size968) = iprot.readListBegin()
-          for _i972 in xrange(_size968):
-            _elem973 = SQLDefaultConstraint()
-            _elem973.read(iprot)
-            self.defaultConstraints.append(_elem973)
+          (_etype985, _size982) = iprot.readListBegin()
+          for _i986 in xrange(_size982):
+            _elem987 = SQLDefaultConstraint()
+            _elem987.read(iprot)
+            self.defaultConstraints.append(_elem987)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype977, _size974) = iprot.readListBegin()
-          for _i978 in xrange(_size974):
-            _elem979 = SQLCheckConstraint()
-            _elem979.read(iprot)
-            self.checkConstraints.append(_elem979)
+          (_etype991, _size988) = iprot.readListBegin()
+          for _i992 in xrange(_size988):
+            _elem993 = SQLCheckConstraint()
+            _elem993.read(iprot)
+            self.checkConstraints.append(_elem993)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19235,43 +19303,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter980 in self.primaryKeys:
-        iter980.write(oprot)
+      for iter994 in self.primaryKeys:
+        iter994.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter981 in self.foreignKeys:
-        iter981.write(oprot)
+      for iter995 in self.foreignKeys:
+        iter995.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter982 in self.uniqueConstraints:
-        iter982.write(oprot)
+      for iter996 in self.uniqueConstraints:
+        iter996.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter983 in self.notNullConstraints:
-        iter983.write(oprot)
+      for iter997 in self.notNullConstraints:
+        iter997.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter984 in self.defaultConstraints:
-        iter984.write(oprot)
+      for iter998 in self.defaultConstraints:
+        iter998.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter985 in self.checkConstraints:
-        iter985.write(oprot)
+      for iter999 in self.checkConstraints:
+        iter999.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20831,10 +20899,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype989, _size986) = iprot.readListBegin()
-          for _i990 in xrange(_size986):
-            _elem991 = iprot.readString()
-            self.partNames.append(_elem991)
+          (_etype1003, _size1000) = iprot.readListBegin()
+          for _i1004 in xrange(_size1000):
+            _elem1005 = iprot.readString()
+            self.partNames.append(_elem1005)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20859,8 +20927,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter992 in self.partNames:
-        oprot.writeString(iter992)
+      for iter1006 in self.partNames:
+        oprot.writeString(iter1006)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21205,10 +21273,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype996, _size993) = iprot.readListBegin()
-          for _i997 in xrange(_size993):
-            _elem998 = iprot.readString()
-            self.success.append(_elem998)
+          (_etype1010, _size1007) = iprot.readListBegin()
+          for _i1011 in xrange(_size1007):
+            _elem1012 = iprot.readString()
+            self.success.append(_elem1012)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21231,8 +21299,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 iter999 in self.success:
-        oprot.writeString(iter999)
+      for iter1013 in self.success:
+        oprot.writeString(iter1013)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21382,10 +21450,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1003, _size1000) = iprot.readListBegin()
-          for _i1004 in xrange(_size1000):
-            _elem1005 = iprot.readString()
-            self.success.append(_elem1005)
+          (_etype1017, _size1014) = iprot.readListBegin()
+          for _i1018 in xrange(_size1014):
+            _elem1019 = iprot.readString()
+            self.success.append(_elem1019)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21408,8 +21476,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1006 in self.success:
-        oprot.writeString(iter1006)
+      for iter1020 in self.success:
+        oprot.writeString(iter1020)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21533,10 +21601,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1010, _size1007) = iprot.readListBegin()
-          for _i1011 in xrange(_size1007):
-            _elem1012 = iprot.readString()
-            self.success.append(_elem1012)
+          (_etype1024, _size1021) = iprot.readListBegin()
+          for _i1025 in xrange(_size1021):
+            _elem1026 = iprot.readString()
+            self.success.append(_elem1026)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21559,8 +21627,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1013 in self.success:
-        oprot.writeString(iter1013)
+      for iter1027 in self.success:
+        oprot.writeString(iter1027)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21633,10 +21701,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype1017, _size1014) = iprot.readListBegin()
-          for _i1018 in xrange(_size1014):
-            _elem1019 = iprot.readString()
-            self.tbl_types.append(_elem1019)
+          (_etype1031, _size1028) = iprot.readListBegin()
+          for _i1032 in xrange(_size1028):
+            _elem1033 = iprot.readString()
+            self.tbl_types.append(_elem1033)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21661,8 +21729,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter1020 in self.tbl_types:
-        oprot.writeString(iter1020)
+      for iter1034 in self.tbl_types:
+        oprot.writeString(iter1034)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21718,11 +21786,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1024, _size1021) = iprot.readListBegin()
-          for _i1025 in xrange(_size1021):
-            _elem1026 = TableMeta()
-            _elem1026.read(iprot)
-            self.success.append(_elem1026)
+          (_etype1038, _size1035) = iprot.readListBegin()
+          for _i1039 in xrange(_size1035):
+            _elem1040 = TableMeta()
+            _elem1040.read(iprot)
+            self.success.append(_elem1040)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21745,8 +21813,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1027 in self.success:
-        iter1027.write(oprot)
+      for iter1041 in self.success:
+        iter1041.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21870,10 +21938,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1031, _size1028) = iprot.readListBegin()
-          for _i1032 in xrange(_size1028):
-            _elem1033 = iprot.readString()
-            self.success.append(_elem1033)
+          (_etype1045, _size1042) = iprot.readListBegin()
+          for _i1046 in xrange(_size1042):
+            _elem1047 = iprot.readString()
+            self.success.append(_elem1047)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21896,8 +21964,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 iter1034 in self.success:
-        oprot.writeString(iter1034)
+      for iter1048 in self.success:
+        oprot.writeString(iter1048)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22133,10 +22201,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype1038, _size1035) = iprot.readListBegin()
-          for _i1039 in xrange(_size1035):
-            _elem1040 = iprot.readString()
-            self.tbl_names.append(_elem1040)
+          (_etype1052, _size1049) = iprot.readListBegin()
+          for _i1053 in xrange(_size1049):
+            _elem1054 = iprot.readString()
+            self.tbl_names.append(_elem1054)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22157,8 +22225,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 iter1041 in self.tbl_names:
-        oprot.writeString(iter1041)
+      for iter1055 in self.tbl_names:
+        oprot.writeString(iter1055)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22210,11 +22278,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1045, _size1042) = iprot.readListBegin()
-          for _i1046 in xrange(_size1042):
-            _elem1047 = Table()
-            _elem1047.read(iprot)
-            self.success.append(_elem1047)
+          (_etype1059, _size1056) = iprot.readListBegin()
+          for _i1060 in xrange(_size1056):
+            _elem1061 = Table()
+            _elem1061.read(iprot)
+            self.success.append(_elem1061)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22231,8 +22299,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 iter1048 in self.success:
-        iter1048.write(oprot)
+      for iter1062 in self.success:
+        iter1062.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23100,10 +23168,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1052, _size1049) = iprot.readListBegin()
-          for _i1053 in xrange(_size1049):
-            _elem1054 = iprot.readString()
-            self.success.append(_elem1054)
+          (_etype1066, _size1063) = iprot.readListBegin()
+          for _i1067 in xrange(_size1063):
+            _elem1068 = iprot.readString()
+            self.success.append(_elem1068)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23138,8 +23206,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 iter1055 in self.success:
-        oprot.writeString(iter1055)
+      for iter1069 in self.success:
+        oprot.writeString(iter1069)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24268,11 +24336,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1059, _size1056) = iprot.readListBegin()
-          for _i1060 in xrange(_size1056):
-            _elem1061 = Partition()
-            _elem1061.read(iprot)
-            self.new_parts.append(_elem1061)
+          (_etype1073, _size1070) = iprot.readListBegin()
+          for _i1074 in xrange(_size1070):
+            _elem1075 = Partition()
+            _elem1075.read(iprot)
+            self.new_parts.append(_elem1075)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24289,8 +24357,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 iter1062 in self.new_parts:
-        iter1062.write(oprot)
+      for iter1076 in self.new_parts:
+        iter1076.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24448,11 +24516,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1066, _size1063) = iprot.readListBegin()
-          for _i1067 in xrange(_size1063):
-            _elem1068 = PartitionSpec()
-            _elem1068.read(iprot)
-            self.new_parts.append(_elem1068)
+          (_etype1080, _size1077) = iprot.readListBegin()
+          for _i1081 in xrange(_size1077):
+            _elem1082 = PartitionSpec()
+            _elem1082.read(iprot)
+            self.new_parts.append(_elem1082)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24469,8 +24537,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 iter1069 in self.new_parts:
-        iter1069.write(oprot)
+      for iter1083 in self.new_parts:
+        iter1083.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24644,10 +24712,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1073, _size1070) = iprot.readListBegin()
-          for _i1074 in xrange(_size1070):
-            _elem1075 = iprot.readString()
-            self.part_vals.append(_elem1075)
+          (_etype1087, _size1084) = iprot.readListBegin()
+          for _i1088 in xrange(_size1084):
+            _elem1089 = iprot.readString()
+            self.part_vals.append(_elem1089)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24672,8 +24740,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 iter1076 in self.part_vals:
-        oprot.writeString(iter1076)
+      for iter1090 in self.part_vals:
+        oprot.writeString(iter1090)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25026,10 +25094,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1080, _size1077) = iprot.readListBegin()
-          for _i1081 in xrange(_size1077):
-            _elem1082 = iprot.readString()
-            self.part_vals.append(_elem1082)
+          (_etype1094, _size1091) = iprot.readListBegin()
+          for _i1095 in xrange(_size1091):
+            _elem1096 = iprot.readString()
+            self.part_vals.append(_elem1096)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25060,8 +25128,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 iter1083 in self.part_vals:
-        oprot.writeString(iter1083)
+      for iter1097 in self.part_vals:
+        oprot.writeString(iter1097)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -25656,10 +25724,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1087, _size1084) = iprot.readListBegin()
-          for _i1088 in xrange(_size1084):
-            _elem1089 = iprot.readString()
-            self.part_vals.append(_elem1089)
+          (_etype1101, _size1098) = iprot.readListBegin()
+          for _i1102 in xrange(_size1098):
+            _elem1103 = iprot.readString()
+            self.part_vals.append(_elem1103)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25689,8 +25757,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 iter1090 in self.part_vals:
-        oprot.writeString(iter1090)
+      for iter1104 in self.part_vals:
+        oprot.writeString(iter1104)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -25863,10 +25931,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1094, _size1091) = iprot.readListBegin()
-          for _i1095 in xrange(_size1091):
-            _elem1096 = iprot.readString()
-            self.part_vals.append(_elem1096)
+          (_etype1108, _size1105) = iprot.readListBegin()
+          for _i1109 in xrange(_size1105):
+            _elem1110 = iprot.readString()
+            self.part_vals.append(_elem1110)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25902,8 +25970,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 iter1097 in self.part_vals:
-        oprot.writeString(iter1097)
+      for iter1111 in self.part_vals:
+        oprot.writeString(iter1111)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -26640,10 +26708,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1101, _size1098) = iprot.readListBegin()
-          for _i1102 in xrange(_size1098):
-            _elem1103 = iprot.readString()
-            self.part_vals.append(_elem1103)
+          (_etype1115, _size1112) = iprot.readListBegin()
+          for _i1116 in xrange(_size1112):
+            _elem1117 = iprot.readString()
+            self.part_vals.append(_elem1117)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26668,8 +26736,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 iter1104 in self.part_vals:
-        oprot.writeString(iter1104)
+      for iter1118 in self.part_vals:
+        oprot.writeString(iter1118)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26828,11 +26896,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1106, _vtype1107, _size1105 ) = iprot.readMapBegin()
-          for _i1109 in xrange(_size1105):
-            _key1110 = iprot.readString()
-            _val1111 = iprot.readString()
-            self.partitionSpecs[_key1110] = _val1111
+          (_ktype1120, _vtype1121, _size1119 ) = iprot.readMapBegin()
+          for _i1123 in xrange(_size1119):
+            _key1124 = iprot.readString()
+            _val1125 = iprot.readString()
+            self.partitionSpecs[_key1124] = _val1125
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -26869,9 +26937,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 kiter1112,viter1113 in self.partitionSpecs.items():
-        oprot.writeString(kiter1112)
-        oprot.writeString(viter1113)
+      for kiter1126,viter1127 in self.partitionSpecs.items():
+        oprot.writeString(kiter1126)
+        oprot.writeString(viter1127)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -27076,11 +27144,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1115, _vtype1116, _size1114 ) = iprot.readMapBegin()
-          for _i1118 in xrange(_size1114):
-            _key1119 = iprot.readString()
-            _val1120 = iprot.readString()
-            self.partitionSpecs[_key1119] = _val1120
+          (_ktype1129, _vtype1130, _size1128 ) = iprot.readMapBegin()
+          for _i1132 in xrange(_size1128):
+            _key1133 = iprot.readString()
+            _val1134 = iprot.readString()
+            self.partitionSpecs[_key1133] = _val1134
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -27117,9 +27185,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 kiter1121,viter1122 in self.partitionSpecs.items():
-        oprot.writeString(kiter1121)
-        oprot.writeString(viter1122)
+      for kiter1135,viter1136 in self.partitionSpecs.items():
+        oprot.writeString(kiter1135)
+        oprot.writeString(viter1136)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -27202,11 +27270,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1126, _size1123) = iprot.readListBegin()
-          for _i1127 in xrange(_size1123):
-            _elem1128 = Partition()
-            _elem1128.read(iprot)
-            self.success.append(_elem1128)
+          (_etype1140, _size1137) = iprot.readListBegin()
+          for _i1141 in xrange(_size1137):
+            _elem1142 = Partition()
+            _elem1142.read(iprot)
+            self.success.append(_elem1142)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27247,8 +27315,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 iter1129 in self.success:
-        iter1129.write(oprot)
+      for iter1143 in self.success:
+        iter1143.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27342,10 +27410,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1133, _size1130) = iprot.readListBegin()
-          for _i1134 in xrange(_size1130):
-            _elem1135 = iprot.readString()
-            self.part_vals.append(_elem1135)
+          (_etype1147, _size1144) = iprot.readListBegin()
+          for _i1148 in xrange(_size1144):
+            _elem1149 = iprot.readString()
+            self.part_vals.append(_elem1149)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27357,10 +27425,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1139, _size1136) = iprot.readListBegin()
-          for _i1140 in xrange(_size1136):
-            _elem1141 = iprot.readString()
-            self.group_names.append(_elem1141)
+          (_etype1153, _size1150) = iprot.readListBegin()
+          for _i1154 in xrange(_size1150):
+            _elem1155 = iprot.readString()
+            self.group_names.append(_elem1155)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27385,8 +27453,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 iter1142 in self.part_vals:
-        oprot.writeString(iter1142)
+      for iter1156 in self.part_vals:
+        oprot.writeString(iter1156)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -27396,8 +27464,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 iter1143 in self.group_names:
-        oprot.writeString(iter1143)
+      for iter1157 in self.group_names:
+        oprot.writeString(iter1157)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27826,11 +27894,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1147, _size1144) = iprot.readListBegin()
-          for _i1148 in xrange(_size1144):
-            _elem1149 = Partition()
-            _elem1149.read(iprot)
-            self.success.append(_elem1149)
+          (_etype1161, _size1158) = iprot.readListBegin()
+          for _i1162 in xrange(_size1158):
+            _elem1163 = Partition()
+            _elem1163.read(iprot)
+            self.success.append(_elem1163)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27859,8 +27927,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 iter1150 in self.success:
-        iter1150.write(oprot)
+      for iter1164 in self.success:
+        iter1164.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27954,10 +28022,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1154, _size1151) = iprot.readListBegin()
-          for _i1155 in xrange(_size1151):
-            _elem1156 = iprot.readString()
-            self.group_names.append(_elem1156)
+          (_etype1168, _size1165) = iprot.readListBegin()
+          for _i1169 in xrange(_size1165):
+            _elem1170 = iprot.readString()
+            self.group_names.append(_elem1170)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27990,8 +28058,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 iter1157 in self.group_names:
-        oprot.writeString(iter1157)
+      for iter1171 in self.group_names:
+        oprot.writeString(iter1171)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28052,11 +28120,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1161, _size1158) = iprot.readListBegin()
-          for _i1162 in xrange(_size1158):
-            _elem1163 = Partition()
-            _elem1163.read(iprot)
-            self.success.append(_elem1163)
+          (_etype1175, _size1172) = iprot.readListBegin()
+          for _i1176 in xrange(_size1172):
+            _elem1177 = Partition()
+            _elem1177.read(iprot)
+            self.success.append(_elem1177)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28085,8 +28153,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 iter1164 in self.success:
-        iter1164.write(oprot)
+      for iter1178 in self.success:
+        iter1178.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28244,11 +28312,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1168, _size1165) = iprot.readListBegin()
-          for _i1169 in xrange(_size1165):
-            _elem1170 = PartitionSpec()
-            _elem1170.read(iprot)
-            self.success.append(_elem1170)
+          (_etype1182, _size1179) = iprot.readListBegin()
+          for _i1183 in xrange(_size1179):
+            _elem1184 = PartitionSpec()
+            _elem1184.read(iprot)
+            self.success.append(_elem1184)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28277,8 +28345,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 iter1171 in self.success:
-        iter1171.write(oprot)
+      for iter1185 in self.success:
+        iter1185.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28436,10 +28504,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1175, _size1172) = iprot.readListBegin()
-          for _i1176 in xrange(_size1172):
-            _elem1177 = iprot.readString()
-            self.success.append(_elem1177)
+          (_etype1189, _size1186) = iprot.readListBegin()
+          for _i1190 in xrange(_size1186):
+            _elem1191 = iprot.readString()
+            self.success.append(_elem1191)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28468,8 +28536,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 iter1178 in self.success:
-        oprot.writeString(iter1178)
+      for iter1192 in self.success:
+        oprot.writeString(iter1192)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28709,10 +28777,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1182, _size1179) = iprot.readListBegin()
-          for _i1183 in xrange(_size1179):
-            _elem1184 = iprot.readString()
-            self.part_vals.append(_elem1184)
+          (_etype1196, _size1193) = iprot.readListBegin()
+          for _i1197 in xrange(_size1193):
+            _elem1198 = iprot.readString()
+            self.part_vals.append(_elem1198)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28742,8 +28810,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 iter1185 in self.part_vals:
-        oprot.writeString(iter1185)
+      for iter1199 in self.part_vals:
+        oprot.writeString(iter1199)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28807,11 +28875,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1189, _size1186) = iprot.readListBegin()
-          for _i1190 in xrange(_size1186):
-            _elem1191 = Partition()
-            _elem1191.read(iprot)
-            self.success.append(_elem1191)
+          (_etype1203, _size1200) = iprot.readListBegin()
+          for _i1204 in xrange(_size1200):
+            _elem1205 = Partition()
+            _elem1205.read(iprot)
+            self.success.append(_elem1205)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28840,8 +28908,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 iter1192 in self.success:
-        iter1192.write(oprot)
+      for iter1206 in self.success:
+        iter1206.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28928,10 +28996,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1196, _size1193) = iprot.readListBegin()
-          for _i1197 in xrange(_size1193):
-            _elem1198 = iprot.readString()
-            self.part_vals.append(_elem1198)
+          (_etype1210, _size1207) = iprot.readListBegin()
+          for _i1211 in xrange(_size1207):
+            _elem1212 = iprot.readString()
+            self.part_vals.append(_elem1212)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28948,10 +29016,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1202, _size1199) = iprot.readListBegin()
-          for _i1203 in xrange(_size1199):
-            _elem1204 = iprot.readString()
-            self.group_names.append(_elem1204)
+          (_etype1216, _size1213) = iprot.readListBegin()
+          for _i1217 in xrange(_size1213):
+            _elem1218 = iprot.readString()
+            self.group_names.append(_elem1218)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28976,8 +29044,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 iter1205 in self.part_vals:
-        oprot.writeString(iter1205)
+      for iter1219 in self.part_vals:
+        oprot.writeString(iter1219)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28991,8 +29059,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 iter1206 in self.group_names:
-        oprot.writeString(iter1206)
+      for iter1220 in self.group_names:
+        oprot.writeString(iter1220)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29054,11 +29122,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1210, _size1207) = iprot.readListBegin()
-          for _i1211 in xrange(_size1207):
-            _elem1212 = Partition()
-            _elem1212.read(iprot)
-            self.success.append(_elem1212)
+          (_etype1224, _size1221) = iprot.readListBegin()
+          for _i1225 in xrange(_size1221):
+            _elem1226 = Partition()
+            _elem1226.read(iprot)
+            self.success.append(_elem1226)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29087,8 +29155,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 iter1213 in self.success:
-        iter1213.write(oprot)
+      for iter1227 in self.success:
+        iter1227.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29169,10 +29237,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1217, _size1214) = iprot.readListBegin()
-          for _i1218 in xrange(_size1214):
-            _elem1219 = iprot.readString()
-            self.part_vals.append(_elem1219)
+          (_etype1231, _size1228) = iprot.readListBegin()
+          for _i1232 in xrange(_size1228):
+            _elem1233 = iprot.readString()
+            self.part_vals.append(_elem1233)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29202,8 +29270,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 iter1220 in self.part_vals:
-        oprot.writeString(iter1220)
+      for iter1234 in self.part_vals:
+        oprot.writeString(iter1234)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -29267,10 +29335,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1224, _size1221) = iprot.readListBegin()
-          for _i1225 in xrange(_size1221):
-            _elem1226 = iprot.readString()
-            self.success.append(_elem1226)
+          (_etype1238, _size1235) = iprot.readListBegin()
+          for _i1239 in xrange(_size1235):
+            _elem1240 = iprot.readString()
+            self.success.append(_elem1240)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29299,8 +29367,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 iter1227 in self.success:
-        oprot.writeString(iter1227)
+      for iter1241 in self.success:
+        oprot.writeString(iter1241)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29471,11 +29539,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1231, _size1228) = iprot.readListBegin()
-          for _i1232 in xrange(_size1228):
-            _elem1233 = Partition()
-            _elem1233.read(iprot)
-            self.success.append(_elem1233)
+          (_etype1245, _size1242) = iprot.readListBegin()
+          for _i1246 in xrange(_size1242):
+            _elem1247 = Partition()
+            _elem1247.read(iprot)
+            self.success.append(_elem1247)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29504,8 +29572,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 iter1234 in self.success:
-        iter1234.write(oprot)
+      for iter1248 in self.success:
+        iter1248.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29676,11 +29744,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1238, _size1235) = iprot.readListBegin()
-          for _i1239 in xrange(_size1235):
-            _elem1240 = PartitionSpec()
-            _elem1240.read(iprot)
-            self.success.append(_elem1240)
+          (_etype1252, _size1249) = iprot.readListBegin()
+          for _i1253 in xrange(_size1249):
+            _elem1254 = PartitionSpec()
+            _elem1254.read(iprot)
+            self.success.append(_elem1254)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29709,8 +29777,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 iter1241 in self.success:
-        iter1241.write(oprot)
+      for iter1255 in self.success:
+        iter1255.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30130,10 +30198,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1245, _size1242) = iprot.readListBegin()
-          for _i1246 in xrange(_size1242):
-            _elem1247 = iprot.readString()
-            self.names.append(_elem1247)
+          (_etype1259, _size1256) = iprot.readListBegin()
+          for _i1260 in xrange(_size1256):
+            _elem1261 = iprot.readString()
+            self.names.append(_elem1261)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30158,8 +30226,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 iter1248 in self.names:
-        oprot.writeString(iter1248)
+      for iter1262 in self.names:
+        oprot.writeString(iter1262)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30218,11 +30286,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1252, _size1249) = iprot.readListBegin()
-          for _i1253 in xrange(_size1249):
-            _elem1254 = Partition()
-            _elem1254.read(iprot)
-            self.success.append(_elem1254)
+          (_etype1266, _size1263) = iprot.readListBegin()
+          for _i1267 in xrange(_size1263):
+            _elem1268 = Partition()
+            _elem1268.read(iprot)
+            self.success.append(_elem1268)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30251,8 +30319,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 iter1255 in self.success:
-        iter1255.write(oprot)
+      for iter1269 in self.success:
+        iter1269.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30288,6 +30356,165 @@ class get_partitions_by_names_result:
   def __ne__(self, other):
     return not (self == other)
 
+class get_partitions_by_names_req_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (GetPartitionsByNamesRequest, GetPartitionsByNamesRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.req = GetPartitionsByNamesRequest()
+          self.req.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_partitions_by_names_req_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class get_partitions_by_names_req_result:
+  """
+  Attributes:
+   - success
+   - o1
+   - o2
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetPartitionsByNamesResult, GetPartitionsByNamesResult.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'o2', (NoSuchObjectException, NoSuchObjectException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, o1=None, o2=None,):
+    self.success = success
+    self.o1 = o1
+    self.o2 = o2
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetPartitionsByNamesResult()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.o2 = NoSuchObjectException()
+          self.o2.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('get_partitions_by_names_req_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o2 is not None:
+      oprot.writeFieldBegin('o2', TType.STRUCT, 2)
+      self.o2.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    value = (value * 31) ^ hash(self.o2)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class alter_partition_args:
   """
   Attributes:
@@ -30502,11 +30729,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1259, _size1256) = iprot.readListBegin()
-          for _i1260 in xrange(_size1256):
-            _elem1261 = Partition()
-            _elem1261.read(iprot)
-            self.new_parts.append(_elem1261)
+          (_etype1273, _size1270) = iprot.readListBegin()
+          for _i1274 in xrange(_size1270):
+            _elem1275 = Partition()
+            _elem1275.read(iprot)
+            self.new_parts.append(_elem1275)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30531,8 +30758,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 iter1262 in self.new_parts:
-        iter1262.write(oprot)
+      for iter1276 in self.new_parts:
+        iter1276.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30685,11 +30912,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1266, _size1263) = iprot.readListBegin()
-          for _i1267 in xrange(_size1263):
-            _elem1268 = Partition()
-            _elem1268.read(iprot)
-            self.new_parts.append(_elem1268)
+          (_etype1280, _size1277) = iprot.readListBegin()
+          for _i1281 in xrange(_size1277):
+            _elem1282 = Partition()
+            _elem1282.read(iprot)
+            self.new_parts.append(_elem1282)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30720,8 +30947,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1269 in self.new_parts:
-        iter1269.write(oprot)
+      for iter1283 in self.new_parts:
+        iter1283.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -31224,10 +31451,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1273, _size1270) = iprot.readListBegin()
-          for _i1274 in xrange(_size1270):
-            _elem1275 = iprot.readString()
-            self.part_vals.append(_elem1275)
+          (_etype1287, _size1284) = iprot.readListBegin()
+          for _i1288 in xrange(_size1284):
+            _elem1289 = iprot.readString()
+            self.part_vals.append(_elem1289)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31258,8 +31485,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 iter1276 in self.part_vals:
-        oprot.writeString(iter1276)
+      for iter1290 in self.part_vals:
+        oprot.writeString(iter1290)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -31560,10 +31787,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1280, _size1277) = iprot.readListBegin()
-          for _i1281 in xrange(_size1277):
-            _elem1282 = iprot.readString()
-            self.part_vals.append(_elem1282)
+          (_etype1294, _size1291) = iprot.readListBegin()
+          for _i1295 in xrange(_size1291):
+            _elem1296 = iprot.readString()
+            self.part_vals.append(_elem1296)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31585,8 +31812,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 iter1283 in self.part_vals:
-        oprot.writeString(iter1283)
+      for iter1297 in self.part_vals:
+        oprot.writeString(iter1297)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -31944,10 +32171,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1287, _size1284) = iprot.readListBegin()
-          for _i1288 in xrange(_size1284):
-            _elem1289 = iprot.readString()
-            self.success.append(_elem1289)
+          (_etype1301, _size1298) = iprot.readListBegin()
+          for _i1302 in xrange(_size1298):
+            _elem1303 = iprot.readString()
+            self.success.append(_elem1303)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31970,8 +32197,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 iter1290 in self.success:
-        oprot.writeString(iter1290)
+      for iter1304 in self.success:
+        oprot.writeString(iter1304)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -32095,11 +32322,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1292, _vtype1293, _size1291 ) = iprot.readMapBegin()
-          for _i1295 in xrange(_size1291):
-            _key1296 = iprot.readString()
-            _val1297 = iprot.readString()
-            self.success[_key1296] = _val1297
+          (_ktype1306, _vtype1307, _size1305 ) = iprot.readMapBegin()
+          for _i1309 in xrange(_size1305):
+            _key1310 = iprot.readString()
+            _val1311 = iprot.readString()
+            self.success[_key1310] = _val1311
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -32122,9 +32349,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 kiter1298,viter1299 in self.success.items():
-        oprot.writeString(kiter1298)
-        oprot.writeString(viter1299)
+      for kiter1312,viter1313 in self.success.items():
+        oprot.writeString(kiter1312)
+        oprot.writeString(viter1313)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -32200,11 +32427,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1301, _vtype1302, _size1300 ) = iprot.readMapBegin()
-          for _i1304 in xrange(_size1300):
-            _key1305 = iprot.readString()
-            _val1306 = iprot.readString()
-            self.part_vals[_key1305] = _val1306
+          (_ktype1315, _vtype1316, _size1314 ) = iprot.readMapBegin()
+          for _i1318 in xrange(_size1314):
+            _key1319 = iprot.readString()
+            _val1320 = iprot.readString()
+            self.part_vals[_key1319] = _val1320
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -32234,9 +32461,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 kiter1307,viter1308 in self.part_vals.items():
-        oprot.writeString(kiter1307)
-        oprot.writeString(viter1308)
+      for kiter1321,viter1322 in self.part_vals.items():
+        oprot.writeString(kiter1321)
+        oprot.writeString(viter1322)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -32450,11 +32677,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1310, _vtype1311, _size1309 ) = iprot.readMapBegin()
-          for _i1313 in xrange(_size1309):
-            _key1314 = iprot.readString()
-            _val1315 = iprot.readString()
-            self.part_vals[_key1314] = _val1315
+          (_ktype1324, _vtype1325, _size1323 ) = iprot.readMapBegin()
+          for _i1327 in xrange(_size1323):
+            _key1328 = iprot.readString()
+            _val1329 = iprot.readString()
+            self.part_vals[_key1328] = _val1329
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -32484,9 +32711,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 kiter1316,viter1317 in self.part_vals.items():
-        oprot.writeString(kiter1316)
-        oprot.writeString(viter1317)
+      for kiter1330,viter1331 in self.part_vals.items():
+        oprot.writeString(kiter1330)
+        oprot.writeString(viter1331)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -36512,10 +36739,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1321, _size1318) = iprot.readListBegin()
-          for _i1322 in xrange(_size1318):
-            _elem1323 = iprot.readString()
-            self.success.append(_elem1323)
+          (_etype1335, _size1332) = iprot.readListBegin()
+          for _i1336 in xrange(_size1332):
+            _elem1337 = iprot.readString()
+            self.success.append(_elem1337)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36538,8 +36765,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 iter1324 in self.success:
-        oprot.writeString(iter1324)
+      for iter1338 in self.success:
+        oprot.writeString(iter1338)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37227,10 +37454,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1328, _size1325) = iprot.readListBegin()
-          for _i1329 in xrange(_size1325):
-            _elem1330 = iprot.readString()
-            self.success.append(_elem1330)
+          (_etype1342, _size1339) = iprot.readListBegin()
+          for _i1343 in xrange(_size1339):
+            _elem1344 = iprot.readString()
+            self.success.append(_elem1344)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37253,8 +37480,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 iter1331 in self.success:
-        oprot.writeString(iter1331)
+      for iter1345 in self.success:
+        oprot.writeString(iter1345)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37768,11 +37995,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1335, _size1332) = iprot.readListBegin()
-          for _i1336 in xrange(_size1332):
-            _elem1337 = Role()
-            _elem1337.read(iprot)
-            self.success.append(_elem1337)
+          (_etype1349, _size1346) = iprot.readListBegin()
+          for _i1350 in xrange(_size1346):
+            _elem1351 = Role()
+            _elem1351.read(iprot)
+            self.success.append(_elem1351)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37795,8 +38022,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 iter1338 in self.success:
-        iter1338.write(oprot)
+      for iter1352 in self.success:
+        iter1352.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -38305,10 +38532,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1342, _size1339) = iprot.readListBegin()
-          for _i1343 in xrange(_size1339):
-            _elem1344 = iprot.readString()
-            self.group_names.append(_elem1344)
+          (_etype1356, _size1353) = iprot.readListBegin()
+          for _i1357 in xrange(_size1353):
+            _elem1358 = iprot.readString()
+            self.group_names.append(_elem1358)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38333,8 +38560,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 iter1345 in self.group_names:
-        oprot.writeString(iter1345)
+      for iter1359 in self.group_names:
+        oprot.writeString(iter1359)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38561,11 +38788,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1349, _size1346) = iprot.readListBegin()
-          for _i1350 in xrange(_size1346):
-            _elem1351 = HiveObjectPrivilege()
-            _elem1351.read(iprot)
-            self.success.append(_elem1351)
+          (_etype1363, _size1360) = iprot.readListBegin()
+          for _i1364 in xrange(_size1360):
+            _elem1365 = HiveObjectPrivilege()
+            _elem1365.read(iprot)
+            self.success.append(_elem1365)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38588,8 +38815,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 iter1352 in self.success:
-        iter1352.write(oprot)
+      for iter1366 in self.success:
+        iter1366.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -39259,10 +39486,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1356, _size1353) = iprot.readListBegin()
-          for _i1357 in xrange(_size1353):
-            _elem1358 = iprot.readString()
-            self.group_names.append(_elem1358)
+          (_etype1370, _size1367) = iprot.readListBegin()
+          for _i1371 in xrange(_size1367):
+            _elem1372 = iprot.readString()
+            self.group_names.append(_elem1372)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -39283,8 +39510,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 iter1359 in self.group_names:
-        oprot.writeString(iter1359)
+      for iter1373 in self.group_names:
+        oprot.writeString(iter1373)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -39339,10 +39566,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1363, _size1360) = iprot.readListBegin()
-          for _i1364 in xrange(_size1360):
-            _elem1365 = iprot.readString()
-            self.success.append(_elem1365)
+          (_etype1377, _size1374) = iprot.readListBegin()
+          for _i1378 in xrange(_size1374):
+            _elem1379 = iprot.readString()
+            self.success.append(_elem1379)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -39365,8 +39592,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 iter1366 in self.success:
-        oprot.writeString(iter1366)
+      for iter1380 in self.success:
+        oprot.writeString(iter1380)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -40298,10 +40525,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1370, _size1367) = iprot.readListBegin()
-          for _i1371 in xrange(_size1367):
-            _elem1372 = iprot.readString()
-            self.success.append(_elem1372)
+          (_etype1384, _size1381) = iprot.readListBegin()
+          for _i1385 in xrange(_size1381):
+            _elem1386 = iprot.readString()
+            self.success.append(_elem1386)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -40318,8 +40545,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1373 in self.success:
-        oprot.writeString(iter1373)
+      for iter1387 in self.success:
+        oprot.writeString(iter1387)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -40846,10 +41073,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1377, _size1374) = iprot.readListBegin()
-          for _i1378 in xrange(_size1374):
-            _elem1379 = iprot.readString()
-            self.success.append(_elem1379)
+          (_etype1391, _size1388) = iprot.readListBegin()
+          for _i1392 in xrange(_size1388):
+            _elem1393 = iprot.readString()
+            self.success.append(_elem1393)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -40866,8 +41093,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1380 in self.success:
-        oprot.writeString(iter1380)
+      for iter1394 in self.success:
+        oprot.writeString(iter1394)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -43880,10 +44107,10 @@ class find_columns_with_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1384, _size1381) = iprot.readListBegin()
-          for _i1385 in xrange(_size1381):
-            _elem1386 = iprot.readString()
-            self.success.append(_elem1386)
+          (_etype1398, _size1395) = iprot.readListBegin()
+          for _i1399 in xrange(_size1395):
+            _elem1400 = iprot.readString()
+            self.success.append(_elem1400)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -43900,8 +44127,8 @@ class find_columns_with_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1387 in self.success:
-        oprot.writeString(iter1387)
+      for iter1401 in self.success:
+        oprot.writeString(iter1401)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -50211,11 +50438,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1391, _size1388) = iprot.readListBegin()
-          for _i1392 in xrange(_size1388):
-            _elem1393 = SchemaVersion()
-            _elem1393.read(iprot)
-            self.success.append(_elem1393)
+          (_etype1405, _size1402) = iprot.readListBegin()
+          for _i1406 in xrange(_size1402):
+            _elem1407 = SchemaVersion()
+            _elem1407.read(iprot)
+            self.success.append(_elem1407)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -50244,8 +50471,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1394 in self.success:
-        iter1394.write(oprot)
+      for iter1408 in self.success:
+        iter1408.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -51720,11 +51947,11 @@ class get_runtime_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1398, _size1395) = iprot.readListBegin()
-          for _i1399 in xrange(_size1395):
-            _elem1400 = RuntimeStat()
-            _elem1400.read(iprot)
-            self.success.append(_elem1400)
+          (_etype1412, _size1409) = iprot.readListBegin()
+          for _i1413 in xrange(_size1409):
+            _elem1414 = RuntimeStat()
+            _elem1414.read(iprot)
+            self.success.append(_elem1414)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -51747,8 +51974,8 @@ class get_runtime_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1401 in self.success:
-        iter1401.write(oprot)
+      for iter1415 in self.success:
+        iter1415.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:


[10/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
index f7d9ed2..70b6e92 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
@@ -877,14 +877,14 @@ import org.slf4j.LoggerFactory;
           case 4: // PARTITIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list968 = iprot.readListBegin();
-                struct.partitions = new ArrayList<Partition>(_list968.size);
-                Partition _elem969;
-                for (int _i970 = 0; _i970 < _list968.size; ++_i970)
+                org.apache.thrift.protocol.TList _list984 = iprot.readListBegin();
+                struct.partitions = new ArrayList<Partition>(_list984.size);
+                Partition _elem985;
+                for (int _i986 = 0; _i986 < _list984.size; ++_i986)
                 {
-                  _elem969 = new Partition();
-                  _elem969.read(iprot);
-                  struct.partitions.add(_elem969);
+                  _elem985 = new Partition();
+                  _elem985.read(iprot);
+                  struct.partitions.add(_elem985);
                 }
                 iprot.readListEnd();
               }
@@ -952,9 +952,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-          for (Partition _iter971 : struct.partitions)
+          for (Partition _iter987 : struct.partitions)
           {
-            _iter971.write(oprot);
+            _iter987.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1000,9 +1000,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tableName);
       {
         oprot.writeI32(struct.partitions.size());
-        for (Partition _iter972 : struct.partitions)
+        for (Partition _iter988 : struct.partitions)
         {
-          _iter972.write(oprot);
+          _iter988.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -1041,14 +1041,14 @@ import org.slf4j.LoggerFactory;
       struct.tableName = iprot.readString();
       struct.setTableNameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list973 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.partitions = new ArrayList<Partition>(_list973.size);
-        Partition _elem974;
-        for (int _i975 = 0; _i975 < _list973.size; ++_i975)
+        org.apache.thrift.protocol.TList _list989 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.partitions = new ArrayList<Partition>(_list989.size);
+        Partition _elem990;
+        for (int _i991 = 0; _i991 < _list989.size; ++_i991)
         {
-          _elem974 = new Partition();
-          _elem974.read(iprot);
-          struct.partitions.add(_elem974);
+          _elem990 = new Partition();
+          _elem990.read(iprot);
+          struct.partitions.add(_elem990);
         }
       }
       struct.setPartitionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index f4e3d6b..1447bb4 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list848.size);
-                long _elem849;
-                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
+                org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list864.size);
+                long _elem865;
+                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
                 {
-                  _elem849 = iprot.readI64();
-                  struct.fileIds.add(_elem849);
+                  _elem865 = iprot.readI64();
+                  struct.fileIds.add(_elem865);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter851 : struct.fileIds)
+          for (long _iter867 : struct.fileIds)
           {
-            oprot.writeI64(_iter851);
+            oprot.writeI64(_iter867);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter852 : struct.fileIds)
+        for (long _iter868 : struct.fileIds)
         {
-          oprot.writeI64(_iter852);
+          oprot.writeI64(_iter868);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list853.size);
-        long _elem854;
-        for (int _i855 = 0; _i855 < _list853.size; ++_i855)
+        org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list869.size);
+        long _elem870;
+        for (int _i871 = 0; _i871 < _list869.size; ++_i871)
         {
-          _elem854 = iprot.readI64();
-          struct.fileIds.add(_elem854);
+          _elem870 = iprot.readI64();
+          struct.fileIds.add(_elem870);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 2b39444..89c12d9 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list864.size);
-                ClientCapability _elem865;
-                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
+                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list880.size);
+                ClientCapability _elem881;
+                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
                 {
-                  _elem865 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem865);
+                  _elem881 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem881);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter867 : struct.values)
+          for (ClientCapability _iter883 : struct.values)
           {
-            oprot.writeI32(_iter867.getValue());
+            oprot.writeI32(_iter883.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter868 : struct.values)
+        for (ClientCapability _iter884 : struct.values)
         {
-          oprot.writeI32(_iter868.getValue());
+          oprot.writeI32(_iter884.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list869.size);
-        ClientCapability _elem870;
-        for (int _i871 = 0; _i871 < _list869.size; ++_i871)
+        org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list885.size);
+        ClientCapability _elem886;
+        for (int _i887 = 0; _i887 < _list885.size; ++_i887)
         {
-          _elem870 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem870);
+          _elem886 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem886);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
index e6b0fd5..71a3001 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
@@ -698,14 +698,14 @@ import org.slf4j.LoggerFactory;
           case 3: // WRITE_EVENT_INFOS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
-                struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list602.size);
-                WriteEventInfo _elem603;
-                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
+                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
+                struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list618.size);
+                WriteEventInfo _elem619;
+                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
                 {
-                  _elem603 = new WriteEventInfo();
-                  _elem603.read(iprot);
-                  struct.writeEventInfos.add(_elem603);
+                  _elem619 = new WriteEventInfo();
+                  _elem619.read(iprot);
+                  struct.writeEventInfos.add(_elem619);
                 }
                 iprot.readListEnd();
               }
@@ -760,9 +760,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WRITE_EVENT_INFOS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.writeEventInfos.size()));
-            for (WriteEventInfo _iter605 : struct.writeEventInfos)
+            for (WriteEventInfo _iter621 : struct.writeEventInfos)
             {
-              _iter605.write(oprot);
+              _iter621.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -821,9 +821,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetWriteEventInfos()) {
         {
           oprot.writeI32(struct.writeEventInfos.size());
-          for (WriteEventInfo _iter606 : struct.writeEventInfos)
+          for (WriteEventInfo _iter622 : struct.writeEventInfos)
           {
-            _iter606.write(oprot);
+            _iter622.write(oprot);
           }
         }
       }
@@ -847,14 +847,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list607.size);
-          WriteEventInfo _elem608;
-          for (int _i609 = 0; _i609 < _list607.size; ++_i609)
+          org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list623.size);
+          WriteEventInfo _elem624;
+          for (int _i625 = 0; _i625 < _list623.size; ++_i625)
           {
-            _elem608 = new WriteEventInfo();
-            _elem608.read(iprot);
-            struct.writeEventInfos.add(_elem608);
+            _elem624 = new WriteEventInfo();
+            _elem624.read(iprot);
+            struct.writeEventInfos.add(_elem624);
           }
         }
         struct.setWriteEventInfosIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 4155d4f..214d65e 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map706 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map706.size);
-                String _key707;
-                String _val708;
-                for (int _i709 = 0; _i709 < _map706.size; ++_i709)
+                org.apache.thrift.protocol.TMap _map722 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map722.size);
+                String _key723;
+                String _val724;
+                for (int _i725 = 0; _i725 < _map722.size; ++_i725)
                 {
-                  _key707 = iprot.readString();
-                  _val708 = iprot.readString();
-                  struct.properties.put(_key707, _val708);
+                  _key723 = iprot.readString();
+                  _val724 = iprot.readString();
+                  struct.properties.put(_key723, _val724);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter710 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter726 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter710.getKey());
-              oprot.writeString(_iter710.getValue());
+              oprot.writeString(_iter726.getKey());
+              oprot.writeString(_iter726.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter711 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter727 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter711.getKey());
-            oprot.writeString(_iter711.getValue());
+            oprot.writeString(_iter727.getKey());
+            oprot.writeString(_iter727.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map712 = 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*_map712.size);
-          String _key713;
-          String _val714;
-          for (int _i715 = 0; _i715 < _map712.size; ++_i715)
+          org.apache.thrift.protocol.TMap _map728 = 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*_map728.size);
+          String _key729;
+          String _val730;
+          for (int _i731 = 0; _i731 < _map728.size; ++_i731)
           {
-            _key713 = iprot.readString();
-            _val714 = iprot.readString();
-            struct.properties.put(_key713, _val714);
+            _key729 = iprot.readString();
+            _val730 = iprot.readString();
+            struct.properties.put(_key729, _val730);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index 9595a5d..2baea57 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -792,13 +792,13 @@ import org.slf4j.LoggerFactory;
           case 4: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set732 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set732.size);
-                String _elem733;
-                for (int _i734 = 0; _i734 < _set732.size; ++_i734)
+                org.apache.thrift.protocol.TSet _set748 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set748.size);
+                String _elem749;
+                for (int _i750 = 0; _i750 < _set748.size; ++_i750)
                 {
-                  _elem733 = iprot.readString();
-                  struct.tablesUsed.add(_elem733);
+                  _elem749 = iprot.readString();
+                  struct.tablesUsed.add(_elem749);
                 }
                 iprot.readSetEnd();
               }
@@ -855,9 +855,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter735 : struct.tablesUsed)
+          for (String _iter751 : struct.tablesUsed)
           {
-            oprot.writeString(_iter735);
+            oprot.writeString(_iter751);
           }
           oprot.writeSetEnd();
         }
@@ -897,9 +897,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter736 : struct.tablesUsed)
+        for (String _iter752 : struct.tablesUsed)
         {
-          oprot.writeString(_iter736);
+          oprot.writeString(_iter752);
         }
       }
       BitSet optionals = new BitSet();
@@ -928,13 +928,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set737 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set737.size);
-        String _elem738;
-        for (int _i739 = 0; _i739 < _set737.size; ++_i739)
+        org.apache.thrift.protocol.TSet _set753 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set753.size);
+        String _elem754;
+        for (int _i755 = 0; _i755 < _set753.size; ++_i755)
         {
-          _elem738 = iprot.readString();
-          struct.tablesUsed.add(_elem738);
+          _elem754 = iprot.readString();
+          struct.tablesUsed.add(_elem754);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index 42073db..f2fffc6 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list960.size);
-                SchemaVersionDescriptor _elem961;
-                for (int _i962 = 0; _i962 < _list960.size; ++_i962)
+                org.apache.thrift.protocol.TList _list976 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list976.size);
+                SchemaVersionDescriptor _elem977;
+                for (int _i978 = 0; _i978 < _list976.size; ++_i978)
                 {
-                  _elem961 = new SchemaVersionDescriptor();
-                  _elem961.read(iprot);
-                  struct.schemaVersions.add(_elem961);
+                  _elem977 = new SchemaVersionDescriptor();
+                  _elem977.read(iprot);
+                  struct.schemaVersions.add(_elem977);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter963 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter979 : struct.schemaVersions)
           {
-            _iter963.write(oprot);
+            _iter979.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter964 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter980 : struct.schemaVersions)
           {
-            _iter964.write(oprot);
+            _iter980.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list965.size);
-          SchemaVersionDescriptor _elem966;
-          for (int _i967 = 0; _i967 < _list965.size; ++_i967)
+          org.apache.thrift.protocol.TList _list981 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list981.size);
+          SchemaVersionDescriptor _elem982;
+          for (int _i983 = 0; _i983 < _list981.size; ++_i983)
           {
-            _elem966 = new SchemaVersionDescriptor();
-            _elem966.read(iprot);
-            struct.schemaVersions.add(_elem966);
+            _elem982 = new SchemaVersionDescriptor();
+            _elem982.read(iprot);
+            struct.schemaVersions.add(_elem982);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index dd6658d..8eb03ad 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -794,13 +794,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list780.size);
-                String _elem781;
-                for (int _i782 = 0; _i782 < _list780.size; ++_i782)
+                org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list796.size);
+                String _elem797;
+                for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                 {
-                  _elem781 = iprot.readString();
-                  struct.partitionVals.add(_elem781);
+                  _elem797 = iprot.readString();
+                  struct.partitionVals.add(_elem797);
                 }
                 iprot.readListEnd();
               }
@@ -857,9 +857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter783 : struct.partitionVals)
+            for (String _iter799 : struct.partitionVals)
             {
-              oprot.writeString(_iter783);
+              oprot.writeString(_iter799);
             }
             oprot.writeListEnd();
           }
@@ -915,9 +915,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter784 : struct.partitionVals)
+          for (String _iter800 : struct.partitionVals)
           {
-            oprot.writeString(_iter784);
+            oprot.writeString(_iter800);
           }
         }
       }
@@ -945,13 +945,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list785.size);
-          String _elem786;
-          for (int _i787 = 0; _i787 < _list785.size; ++_i787)
+          org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list801.size);
+          String _elem802;
+          for (int _i803 = 0; _i803 < _list801.size; ++_i803)
           {
-            _elem786 = iprot.readString();
-            struct.partitionVals.add(_elem786);
+            _elem802 = iprot.readString();
+            struct.partitionVals.add(_elem802);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
index ce0feb9..6d4aaf1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Function.java
@@ -1079,14 +1079,14 @@ import org.slf4j.LoggerFactory;
           case 8: // RESOURCE_URIS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list554 = iprot.readListBegin();
-                struct.resourceUris = new ArrayList<ResourceUri>(_list554.size);
-                ResourceUri _elem555;
-                for (int _i556 = 0; _i556 < _list554.size; ++_i556)
+                org.apache.thrift.protocol.TList _list570 = iprot.readListBegin();
+                struct.resourceUris = new ArrayList<ResourceUri>(_list570.size);
+                ResourceUri _elem571;
+                for (int _i572 = 0; _i572 < _list570.size; ++_i572)
                 {
-                  _elem555 = new ResourceUri();
-                  _elem555.read(iprot);
-                  struct.resourceUris.add(_elem555);
+                  _elem571 = new ResourceUri();
+                  _elem571.read(iprot);
+                  struct.resourceUris.add(_elem571);
                 }
                 iprot.readListEnd();
               }
@@ -1153,9 +1153,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(RESOURCE_URIS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourceUris.size()));
-          for (ResourceUri _iter557 : struct.resourceUris)
+          for (ResourceUri _iter573 : struct.resourceUris)
           {
-            _iter557.write(oprot);
+            _iter573.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1238,9 +1238,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourceUris()) {
         {
           oprot.writeI32(struct.resourceUris.size());
-          for (ResourceUri _iter558 : struct.resourceUris)
+          for (ResourceUri _iter574 : struct.resourceUris)
           {
-            _iter558.write(oprot);
+            _iter574.write(oprot);
           }
         }
       }
@@ -1283,14 +1283,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list559 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourceUris = new ArrayList<ResourceUri>(_list559.size);
-          ResourceUri _elem560;
-          for (int _i561 = 0; _i561 < _list559.size; ++_i561)
+          org.apache.thrift.protocol.TList _list575 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourceUris = new ArrayList<ResourceUri>(_list575.size);
+          ResourceUri _elem576;
+          for (int _i577 = 0; _i577 < _list575.size; ++_i577)
           {
-            _elem560 = new ResourceUri();
-            _elem560.read(iprot);
-            struct.resourceUris.add(_elem560);
+            _elem576 = new ResourceUri();
+            _elem576.read(iprot);
+            struct.resourceUris.add(_elem576);
           }
         }
         struct.setResourceUrisIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 68146e4..80ade22 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list856.size);
-                Function _elem857;
-                for (int _i858 = 0; _i858 < _list856.size; ++_i858)
+                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list872.size);
+                Function _elem873;
+                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
                 {
-                  _elem857 = new Function();
-                  _elem857.read(iprot);
-                  struct.functions.add(_elem857);
+                  _elem873 = new Function();
+                  _elem873.read(iprot);
+                  struct.functions.add(_elem873);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter859 : struct.functions)
+            for (Function _iter875 : struct.functions)
             {
-              _iter859.write(oprot);
+              _iter875.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter860 : struct.functions)
+          for (Function _iter876 : struct.functions)
           {
-            _iter860.write(oprot);
+            _iter876.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list861.size);
-          Function _elem862;
-          for (int _i863 = 0; _i863 < _list861.size; ++_i863)
+          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list877.size);
+          Function _elem878;
+          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
           {
-            _elem862 = new Function();
-            _elem862.read(iprot);
-            struct.functions.add(_elem862);
+            _elem878 = new Function();
+            _elem878.read(iprot);
+            struct.functions.add(_elem878);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index ee535a0..8392709 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list806 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list806.size);
-                long _elem807;
-                for (int _i808 = 0; _i808 < _list806.size; ++_i808)
+                org.apache.thrift.protocol.TList _list822 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list822.size);
+                long _elem823;
+                for (int _i824 = 0; _i824 < _list822.size; ++_i824)
                 {
-                  _elem807 = iprot.readI64();
-                  struct.fileIds.add(_elem807);
+                  _elem823 = iprot.readI64();
+                  struct.fileIds.add(_elem823);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter809 : struct.fileIds)
+          for (long _iter825 : struct.fileIds)
           {
-            oprot.writeI64(_iter809);
+            oprot.writeI64(_iter825);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter810 : struct.fileIds)
+        for (long _iter826 : struct.fileIds)
         {
-          oprot.writeI64(_iter810);
+          oprot.writeI64(_iter826);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list811 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list811.size);
-        long _elem812;
-        for (int _i813 = 0; _i813 < _list811.size; ++_i813)
+        org.apache.thrift.protocol.TList _list827 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list827.size);
+        long _elem828;
+        for (int _i829 = 0; _i829 < _list827.size; ++_i829)
         {
-          _elem812 = iprot.readI64();
-          struct.fileIds.add(_elem812);
+          _elem828 = iprot.readI64();
+          struct.fileIds.add(_elem828);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index 71e92b6..bf2c384 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map796 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map796.size);
-                long _key797;
-                MetadataPpdResult _val798;
-                for (int _i799 = 0; _i799 < _map796.size; ++_i799)
+                org.apache.thrift.protocol.TMap _map812 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map812.size);
+                long _key813;
+                MetadataPpdResult _val814;
+                for (int _i815 = 0; _i815 < _map812.size; ++_i815)
                 {
-                  _key797 = iprot.readI64();
-                  _val798 = new MetadataPpdResult();
-                  _val798.read(iprot);
-                  struct.metadata.put(_key797, _val798);
+                  _key813 = iprot.readI64();
+                  _val814 = new MetadataPpdResult();
+                  _val814.read(iprot);
+                  struct.metadata.put(_key813, _val814);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter800 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter816 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter800.getKey());
-            _iter800.getValue().write(oprot);
+            oprot.writeI64(_iter816.getKey());
+            _iter816.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter801 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter817 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter801.getKey());
-          _iter801.getValue().write(oprot);
+          oprot.writeI64(_iter817.getKey());
+          _iter817.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map802 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map802.size);
-        long _key803;
-        MetadataPpdResult _val804;
-        for (int _i805 = 0; _i805 < _map802.size; ++_i805)
+        org.apache.thrift.protocol.TMap _map818 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map818.size);
+        long _key819;
+        MetadataPpdResult _val820;
+        for (int _i821 = 0; _i821 < _map818.size; ++_i821)
         {
-          _key803 = iprot.readI64();
-          _val804 = new MetadataPpdResult();
-          _val804.read(iprot);
-          struct.metadata.put(_key803, _val804);
+          _key819 = iprot.readI64();
+          _val820 = new MetadataPpdResult();
+          _val820.read(iprot);
+          struct.metadata.put(_key819, _val820);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index 0ea6ef5..2719d51 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list824.size);
-                long _elem825;
-                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
+                org.apache.thrift.protocol.TList _list840 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list840.size);
+                long _elem841;
+                for (int _i842 = 0; _i842 < _list840.size; ++_i842)
                 {
-                  _elem825 = iprot.readI64();
-                  struct.fileIds.add(_elem825);
+                  _elem841 = iprot.readI64();
+                  struct.fileIds.add(_elem841);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter827 : struct.fileIds)
+          for (long _iter843 : struct.fileIds)
           {
-            oprot.writeI64(_iter827);
+            oprot.writeI64(_iter843);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter828 : struct.fileIds)
+        for (long _iter844 : struct.fileIds)
         {
-          oprot.writeI64(_iter828);
+          oprot.writeI64(_iter844);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list829.size);
-        long _elem830;
-        for (int _i831 = 0; _i831 < _list829.size; ++_i831)
+        org.apache.thrift.protocol.TList _list845 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list845.size);
+        long _elem846;
+        for (int _i847 = 0; _i847 < _list845.size; ++_i847)
         {
-          _elem830 = iprot.readI64();
-          struct.fileIds.add(_elem830);
+          _elem846 = iprot.readI64();
+          struct.fileIds.add(_elem846);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 759b495..798b536 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map814 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map814.size);
-                long _key815;
-                ByteBuffer _val816;
-                for (int _i817 = 0; _i817 < _map814.size; ++_i817)
+                org.apache.thrift.protocol.TMap _map830 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map830.size);
+                long _key831;
+                ByteBuffer _val832;
+                for (int _i833 = 0; _i833 < _map830.size; ++_i833)
                 {
-                  _key815 = iprot.readI64();
-                  _val816 = iprot.readBinary();
-                  struct.metadata.put(_key815, _val816);
+                  _key831 = iprot.readI64();
+                  _val832 = iprot.readBinary();
+                  struct.metadata.put(_key831, _val832);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter818 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter834 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter818.getKey());
-            oprot.writeBinary(_iter818.getValue());
+            oprot.writeI64(_iter834.getKey());
+            oprot.writeBinary(_iter834.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter819 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter835 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter819.getKey());
-          oprot.writeBinary(_iter819.getValue());
+          oprot.writeI64(_iter835.getKey());
+          oprot.writeBinary(_iter835.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map820 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map820.size);
-        long _key821;
-        ByteBuffer _val822;
-        for (int _i823 = 0; _i823 < _map820.size; ++_i823)
+        org.apache.thrift.protocol.TMap _map836 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map836.size);
+        long _key837;
+        ByteBuffer _val838;
+        for (int _i839 = 0; _i839 < _map836.size; ++_i839)
         {
-          _key821 = iprot.readI64();
-          _val822 = iprot.readBinary();
-          struct.metadata.put(_key821, _val822);
+          _key837 = iprot.readI64();
+          _val838 = iprot.readBinary();
+          struct.metadata.put(_key837, _val838);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
index 93ca303..68f7112 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsInfoResponse.java
@@ -447,14 +447,14 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list562 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<TxnInfo>(_list562.size);
-                TxnInfo _elem563;
-                for (int _i564 = 0; _i564 < _list562.size; ++_i564)
+                org.apache.thrift.protocol.TList _list578 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<TxnInfo>(_list578.size);
+                TxnInfo _elem579;
+                for (int _i580 = 0; _i580 < _list578.size; ++_i580)
                 {
-                  _elem563 = new TxnInfo();
-                  _elem563.read(iprot);
-                  struct.open_txns.add(_elem563);
+                  _elem579 = new TxnInfo();
+                  _elem579.read(iprot);
+                  struct.open_txns.add(_elem579);
                 }
                 iprot.readListEnd();
               }
@@ -483,9 +483,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.open_txns.size()));
-          for (TxnInfo _iter565 : struct.open_txns)
+          for (TxnInfo _iter581 : struct.open_txns)
           {
-            _iter565.write(oprot);
+            _iter581.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -511,9 +511,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (TxnInfo _iter566 : struct.open_txns)
+        for (TxnInfo _iter582 : struct.open_txns)
         {
-          _iter566.write(oprot);
+          _iter582.write(oprot);
         }
       }
     }
@@ -524,14 +524,14 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.open_txns = new ArrayList<TxnInfo>(_list567.size);
-        TxnInfo _elem568;
-        for (int _i569 = 0; _i569 < _list567.size; ++_i569)
+        org.apache.thrift.protocol.TList _list583 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.open_txns = new ArrayList<TxnInfo>(_list583.size);
+        TxnInfo _elem584;
+        for (int _i585 = 0; _i585 < _list583.size; ++_i585)
         {
-          _elem568 = new TxnInfo();
-          _elem568.read(iprot);
-          struct.open_txns.add(_elem568);
+          _elem584 = new TxnInfo();
+          _elem584.read(iprot);
+          struct.open_txns.add(_elem584);
         }
       }
       struct.setOpen_txnsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
index c152a0a..a4284a0 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetOpenTxnsResponse.java
@@ -615,13 +615,13 @@ import org.slf4j.LoggerFactory;
           case 2: // OPEN_TXNS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list570 = iprot.readListBegin();
-                struct.open_txns = new ArrayList<Long>(_list570.size);
-                long _elem571;
-                for (int _i572 = 0; _i572 < _list570.size; ++_i572)
+                org.apache.thrift.protocol.TList _list586 = iprot.readListBegin();
+                struct.open_txns = new ArrayList<Long>(_list586.size);
+                long _elem587;
+                for (int _i588 = 0; _i588 < _list586.size; ++_i588)
                 {
-                  _elem571 = iprot.readI64();
-                  struct.open_txns.add(_elem571);
+                  _elem587 = iprot.readI64();
+                  struct.open_txns.add(_elem587);
                 }
                 iprot.readListEnd();
               }
@@ -666,9 +666,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(OPEN_TXNS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.open_txns.size()));
-          for (long _iter573 : struct.open_txns)
+          for (long _iter589 : struct.open_txns)
           {
-            oprot.writeI64(_iter573);
+            oprot.writeI64(_iter589);
           }
           oprot.writeListEnd();
         }
@@ -704,9 +704,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.txn_high_water_mark);
       {
         oprot.writeI32(struct.open_txns.size());
-        for (long _iter574 : struct.open_txns)
+        for (long _iter590 : struct.open_txns)
         {
-          oprot.writeI64(_iter574);
+          oprot.writeI64(_iter590);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -726,13 +726,13 @@ import org.slf4j.LoggerFactory;
       struct.txn_high_water_mark = iprot.readI64();
       struct.setTxn_high_water_markIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list575 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.open_txns = new ArrayList<Long>(_list575.size);
-        long _elem576;
-        for (int _i577 = 0; _i577 < _list575.size; ++_i577)
+        org.apache.thrift.protocol.TList _list591 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.open_txns = new ArrayList<Long>(_list591.size);
+        long _elem592;
+        for (int _i593 = 0; _i593 < _list591.size; ++_i593)
         {
-          _elem576 = iprot.readI64();
-          struct.open_txns.add(_elem576);
+          _elem592 = iprot.readI64();
+          struct.open_txns.add(_elem592);
         }
       }
       struct.setOpen_txnsIsSet(true);


[02/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 06938b4..dee644c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -5142,6 +5142,7 @@ class Partition:
    - catName
    - writeId
    - isStatsCompliant
+   - colStats
   """
 
   thrift_spec = (
@@ -5157,9 +5158,10 @@ class Partition:
     (9, TType.STRING, 'catName', None, None, ), # 9
     (10, TType.I64, 'writeId', None, -1, ), # 10
     (11, TType.BOOL, 'isStatsCompliant', None, None, ), # 11
+    (12, TType.STRUCT, 'colStats', (ColumnStatistics, ColumnStatistics.thrift_spec), None, ), # 12
   )
 
-  def __init__(self, values=None, dbName=None, tableName=None, createTime=None, lastAccessTime=None, sd=None, parameters=None, privileges=None, catName=None, writeId=thrift_spec[10][4], isStatsCompliant=None,):
+  def __init__(self, values=None, dbName=None, tableName=None, createTime=None, lastAccessTime=None, sd=None, parameters=None, privileges=None, catName=None, writeId=thrift_spec[10][4], isStatsCompliant=None, colStats=None,):
     self.values = values
     self.dbName = dbName
     self.tableName = tableName
@@ -5171,6 +5173,7 @@ class Partition:
     self.catName = catName
     self.writeId = writeId
     self.isStatsCompliant = isStatsCompliant
+    self.colStats = colStats
 
   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:
@@ -5249,6 +5252,12 @@ class Partition:
           self.isStatsCompliant = iprot.readBool()
         else:
           iprot.skip(ftype)
+      elif fid == 12:
+        if ftype == TType.STRUCT:
+          self.colStats = ColumnStatistics()
+          self.colStats.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -5310,6 +5319,10 @@ class Partition:
       oprot.writeFieldBegin('isStatsCompliant', TType.BOOL, 11)
       oprot.writeBool(self.isStatsCompliant)
       oprot.writeFieldEnd()
+    if self.colStats is not None:
+      oprot.writeFieldBegin('colStats', TType.STRUCT, 12)
+      self.colStats.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -5330,6 +5343,7 @@ class Partition:
     value = (value * 31) ^ hash(self.catName)
     value = (value * 31) ^ hash(self.writeId)
     value = (value * 31) ^ hash(self.isStatsCompliant)
+    value = (value * 31) ^ hash(self.colStats)
     return value
 
   def __repr__(self):
@@ -11081,6 +11095,198 @@ class PartitionValuesResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class GetPartitionsByNamesRequest:
+  """
+  Attributes:
+   - db_name
+   - tbl_name
+   - names
+   - get_col_stats
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'db_name', None, None, ), # 1
+    (2, TType.STRING, 'tbl_name', None, None, ), # 2
+    (3, TType.LIST, 'names', (TType.STRING,None), None, ), # 3
+    (4, TType.BOOL, 'get_col_stats', None, None, ), # 4
+  )
+
+  def __init__(self, db_name=None, tbl_name=None, names=None, get_col_stats=None,):
+    self.db_name = db_name
+    self.tbl_name = tbl_name
+    self.names = names
+    self.get_col_stats = get_col_stats
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.db_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tbl_name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.names = []
+          (_etype491, _size488) = iprot.readListBegin()
+          for _i492 in xrange(_size488):
+            _elem493 = iprot.readString()
+            self.names.append(_elem493)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.BOOL:
+          self.get_col_stats = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetPartitionsByNamesRequest')
+    if self.db_name is not None:
+      oprot.writeFieldBegin('db_name', TType.STRING, 1)
+      oprot.writeString(self.db_name)
+      oprot.writeFieldEnd()
+    if self.tbl_name is not None:
+      oprot.writeFieldBegin('tbl_name', TType.STRING, 2)
+      oprot.writeString(self.tbl_name)
+      oprot.writeFieldEnd()
+    if self.names is not None:
+      oprot.writeFieldBegin('names', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRING, len(self.names))
+      for iter494 in self.names:
+        oprot.writeString(iter494)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.get_col_stats is not None:
+      oprot.writeFieldBegin('get_col_stats', TType.BOOL, 4)
+      oprot.writeBool(self.get_col_stats)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.db_name is None:
+      raise TProtocol.TProtocolException(message='Required field db_name is unset!')
+    if self.tbl_name is None:
+      raise TProtocol.TProtocolException(message='Required field tbl_name is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.db_name)
+    value = (value * 31) ^ hash(self.tbl_name)
+    value = (value * 31) ^ hash(self.names)
+    value = (value * 31) ^ hash(self.get_col_stats)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetPartitionsByNamesResult:
+  """
+  Attributes:
+   - partitions
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'partitions', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, partitions=None,):
+    self.partitions = partitions
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.partitions = []
+          (_etype498, _size495) = iprot.readListBegin()
+          for _i499 in xrange(_size495):
+            _elem500 = Partition()
+            _elem500.read(iprot)
+            self.partitions.append(_elem500)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetPartitionsByNamesResult')
+    if self.partitions is not None:
+      oprot.writeFieldBegin('partitions', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.partitions))
+      for iter501 in self.partitions:
+        iter501.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.partitions is None:
+      raise TProtocol.TProtocolException(message='Required field partitions is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.partitions)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class ResourceUri:
   """
   Attributes:
@@ -11244,11 +11450,11 @@ class Function:
       elif fid == 8:
         if ftype == TType.LIST:
           self.resourceUris = []
-          (_etype491, _size488) = iprot.readListBegin()
-          for _i492 in xrange(_size488):
-            _elem493 = ResourceUri()
-            _elem493.read(iprot)
-            self.resourceUris.append(_elem493)
+          (_etype505, _size502) = iprot.readListBegin()
+          for _i506 in xrange(_size502):
+            _elem507 = ResourceUri()
+            _elem507.read(iprot)
+            self.resourceUris.append(_elem507)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11298,8 +11504,8 @@ class Function:
     if self.resourceUris is not None:
       oprot.writeFieldBegin('resourceUris', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.resourceUris))
-      for iter494 in self.resourceUris:
-        iter494.write(oprot)
+      for iter508 in self.resourceUris:
+        iter508.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.catName is not None:
@@ -11548,11 +11754,11 @@ class GetOpenTxnsInfoResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype498, _size495) = iprot.readListBegin()
-          for _i499 in xrange(_size495):
-            _elem500 = TxnInfo()
-            _elem500.read(iprot)
-            self.open_txns.append(_elem500)
+          (_etype512, _size509) = iprot.readListBegin()
+          for _i513 in xrange(_size509):
+            _elem514 = TxnInfo()
+            _elem514.read(iprot)
+            self.open_txns.append(_elem514)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11573,8 +11779,8 @@ class GetOpenTxnsInfoResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.open_txns))
-      for iter501 in self.open_txns:
-        iter501.write(oprot)
+      for iter515 in self.open_txns:
+        iter515.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11645,10 +11851,10 @@ class GetOpenTxnsResponse:
       elif fid == 2:
         if ftype == TType.LIST:
           self.open_txns = []
-          (_etype505, _size502) = iprot.readListBegin()
-          for _i506 in xrange(_size502):
-            _elem507 = iprot.readI64()
-            self.open_txns.append(_elem507)
+          (_etype519, _size516) = iprot.readListBegin()
+          for _i520 in xrange(_size516):
+            _elem521 = iprot.readI64()
+            self.open_txns.append(_elem521)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11679,8 +11885,8 @@ class GetOpenTxnsResponse:
     if self.open_txns is not None:
       oprot.writeFieldBegin('open_txns', TType.LIST, 2)
       oprot.writeListBegin(TType.I64, len(self.open_txns))
-      for iter508 in self.open_txns:
-        oprot.writeI64(iter508)
+      for iter522 in self.open_txns:
+        oprot.writeI64(iter522)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.min_open_txn is not None:
@@ -11792,10 +11998,10 @@ class OpenTxnRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.replSrcTxnIds = []
-          (_etype512, _size509) = iprot.readListBegin()
-          for _i513 in xrange(_size509):
-            _elem514 = iprot.readI64()
-            self.replSrcTxnIds.append(_elem514)
+          (_etype526, _size523) = iprot.readListBegin()
+          for _i527 in xrange(_size523):
+            _elem528 = iprot.readI64()
+            self.replSrcTxnIds.append(_elem528)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11837,8 +12043,8 @@ class OpenTxnRequest:
     if self.replSrcTxnIds is not None:
       oprot.writeFieldBegin('replSrcTxnIds', TType.LIST, 6)
       oprot.writeListBegin(TType.I64, len(self.replSrcTxnIds))
-      for iter515 in self.replSrcTxnIds:
-        oprot.writeI64(iter515)
+      for iter529 in self.replSrcTxnIds:
+        oprot.writeI64(iter529)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txn_type is not None:
@@ -11906,10 +12112,10 @@ class OpenTxnsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype519, _size516) = iprot.readListBegin()
-          for _i520 in xrange(_size516):
-            _elem521 = iprot.readI64()
-            self.txn_ids.append(_elem521)
+          (_etype533, _size530) = iprot.readListBegin()
+          for _i534 in xrange(_size530):
+            _elem535 = iprot.readI64()
+            self.txn_ids.append(_elem535)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11926,8 +12132,8 @@ class OpenTxnsResponse:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter522 in self.txn_ids:
-        oprot.writeI64(iter522)
+      for iter536 in self.txn_ids:
+        oprot.writeI64(iter536)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12061,10 +12267,10 @@ class AbortTxnsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.txn_ids = []
-          (_etype526, _size523) = iprot.readListBegin()
-          for _i527 in xrange(_size523):
-            _elem528 = iprot.readI64()
-            self.txn_ids.append(_elem528)
+          (_etype540, _size537) = iprot.readListBegin()
+          for _i541 in xrange(_size537):
+            _elem542 = iprot.readI64()
+            self.txn_ids.append(_elem542)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12081,8 +12287,8 @@ class AbortTxnsRequest:
     if self.txn_ids is not None:
       oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.txn_ids))
-      for iter529 in self.txn_ids:
-        oprot.writeI64(iter529)
+      for iter543 in self.txn_ids:
+        oprot.writeI64(iter543)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12255,11 +12461,11 @@ class CommitTxnRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.writeEventInfos = []
-          (_etype533, _size530) = iprot.readListBegin()
-          for _i534 in xrange(_size530):
-            _elem535 = WriteEventInfo()
-            _elem535.read(iprot)
-            self.writeEventInfos.append(_elem535)
+          (_etype547, _size544) = iprot.readListBegin()
+          for _i548 in xrange(_size544):
+            _elem549 = WriteEventInfo()
+            _elem549.read(iprot)
+            self.writeEventInfos.append(_elem549)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12296,8 +12502,8 @@ class CommitTxnRequest:
     if self.writeEventInfos is not None:
       oprot.writeFieldBegin('writeEventInfos', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.writeEventInfos))
-      for iter536 in self.writeEventInfos:
-        iter536.write(oprot)
+      for iter550 in self.writeEventInfos:
+        iter550.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.keyValue is not None:
@@ -12398,10 +12604,10 @@ class ReplLastIdInfo:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionList = []
-          (_etype540, _size537) = iprot.readListBegin()
-          for _i541 in xrange(_size537):
-            _elem542 = iprot.readString()
-            self.partitionList.append(_elem542)
+          (_etype554, _size551) = iprot.readListBegin()
+          for _i555 in xrange(_size551):
+            _elem556 = iprot.readString()
+            self.partitionList.append(_elem556)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12439,8 +12645,8 @@ class ReplLastIdInfo:
     if self.partitionList is not None:
       oprot.writeFieldBegin('partitionList', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionList))
-      for iter543 in self.partitionList:
-        oprot.writeString(iter543)
+      for iter557 in self.partitionList:
+        oprot.writeString(iter557)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.needUpdateDBReplId is not None:
@@ -12696,10 +12902,10 @@ class ReplTblWriteIdStateRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype547, _size544) = iprot.readListBegin()
-          for _i548 in xrange(_size544):
-            _elem549 = iprot.readString()
-            self.partNames.append(_elem549)
+          (_etype561, _size558) = iprot.readListBegin()
+          for _i562 in xrange(_size558):
+            _elem563 = iprot.readString()
+            self.partNames.append(_elem563)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12736,8 +12942,8 @@ class ReplTblWriteIdStateRequest:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter550 in self.partNames:
-        oprot.writeString(iter550)
+      for iter564 in self.partNames:
+        oprot.writeString(iter564)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12810,10 +13016,10 @@ class GetValidWriteIdsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fullTableNames = []
-          (_etype554, _size551) = iprot.readListBegin()
-          for _i555 in xrange(_size551):
-            _elem556 = iprot.readString()
-            self.fullTableNames.append(_elem556)
+          (_etype568, _size565) = iprot.readListBegin()
+          for _i569 in xrange(_size565):
+            _elem570 = iprot.readString()
+            self.fullTableNames.append(_elem570)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12840,8 +13046,8 @@ class GetValidWriteIdsRequest:
     if self.fullTableNames is not None:
       oprot.writeFieldBegin('fullTableNames', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.fullTableNames))
-      for iter557 in self.fullTableNames:
-        oprot.writeString(iter557)
+      for iter571 in self.fullTableNames:
+        oprot.writeString(iter571)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -12927,10 +13133,10 @@ class TableValidWriteIds:
       elif fid == 3:
         if ftype == TType.LIST:
           self.invalidWriteIds = []
-          (_etype561, _size558) = iprot.readListBegin()
-          for _i562 in xrange(_size558):
-            _elem563 = iprot.readI64()
-            self.invalidWriteIds.append(_elem563)
+          (_etype575, _size572) = iprot.readListBegin()
+          for _i576 in xrange(_size572):
+            _elem577 = iprot.readI64()
+            self.invalidWriteIds.append(_elem577)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12965,8 +13171,8 @@ class TableValidWriteIds:
     if self.invalidWriteIds is not None:
       oprot.writeFieldBegin('invalidWriteIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.invalidWriteIds))
-      for iter564 in self.invalidWriteIds:
-        oprot.writeI64(iter564)
+      for iter578 in self.invalidWriteIds:
+        oprot.writeI64(iter578)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.minOpenWriteId is not None:
@@ -13038,11 +13244,11 @@ class GetValidWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.tblValidWriteIds = []
-          (_etype568, _size565) = iprot.readListBegin()
-          for _i569 in xrange(_size565):
-            _elem570 = TableValidWriteIds()
-            _elem570.read(iprot)
-            self.tblValidWriteIds.append(_elem570)
+          (_etype582, _size579) = iprot.readListBegin()
+          for _i583 in xrange(_size579):
+            _elem584 = TableValidWriteIds()
+            _elem584.read(iprot)
+            self.tblValidWriteIds.append(_elem584)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13059,8 +13265,8 @@ class GetValidWriteIdsResponse:
     if self.tblValidWriteIds is not None:
       oprot.writeFieldBegin('tblValidWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tblValidWriteIds))
-      for iter571 in self.tblValidWriteIds:
-        iter571.write(oprot)
+      for iter585 in self.tblValidWriteIds:
+        iter585.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13136,10 +13342,10 @@ class AllocateTableWriteIdsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.txnIds = []
-          (_etype575, _size572) = iprot.readListBegin()
-          for _i576 in xrange(_size572):
-            _elem577 = iprot.readI64()
-            self.txnIds.append(_elem577)
+          (_etype589, _size586) = iprot.readListBegin()
+          for _i590 in xrange(_size586):
+            _elem591 = iprot.readI64()
+            self.txnIds.append(_elem591)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13151,11 +13357,11 @@ class AllocateTableWriteIdsRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.srcTxnToWriteIdList = []
-          (_etype581, _size578) = iprot.readListBegin()
-          for _i582 in xrange(_size578):
-            _elem583 = TxnToWriteId()
-            _elem583.read(iprot)
-            self.srcTxnToWriteIdList.append(_elem583)
+          (_etype595, _size592) = iprot.readListBegin()
+          for _i596 in xrange(_size592):
+            _elem597 = TxnToWriteId()
+            _elem597.read(iprot)
+            self.srcTxnToWriteIdList.append(_elem597)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13180,8 +13386,8 @@ class AllocateTableWriteIdsRequest:
     if self.txnIds is not None:
       oprot.writeFieldBegin('txnIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.txnIds))
-      for iter584 in self.txnIds:
-        oprot.writeI64(iter584)
+      for iter598 in self.txnIds:
+        oprot.writeI64(iter598)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.replPolicy is not None:
@@ -13191,8 +13397,8 @@ class AllocateTableWriteIdsRequest:
     if self.srcTxnToWriteIdList is not None:
       oprot.writeFieldBegin('srcTxnToWriteIdList', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.srcTxnToWriteIdList))
-      for iter585 in self.srcTxnToWriteIdList:
-        iter585.write(oprot)
+      for iter599 in self.srcTxnToWriteIdList:
+        iter599.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13334,11 +13540,11 @@ class AllocateTableWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnToWriteIds = []
-          (_etype589, _size586) = iprot.readListBegin()
-          for _i590 in xrange(_size586):
-            _elem591 = TxnToWriteId()
-            _elem591.read(iprot)
-            self.txnToWriteIds.append(_elem591)
+          (_etype603, _size600) = iprot.readListBegin()
+          for _i604 in xrange(_size600):
+            _elem605 = TxnToWriteId()
+            _elem605.read(iprot)
+            self.txnToWriteIds.append(_elem605)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13355,8 +13561,8 @@ class AllocateTableWriteIdsResponse:
     if self.txnToWriteIds is not None:
       oprot.writeFieldBegin('txnToWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.txnToWriteIds))
-      for iter592 in self.txnToWriteIds:
-        iter592.write(oprot)
+      for iter606 in self.txnToWriteIds:
+        iter606.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13584,11 +13790,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype596, _size593) = iprot.readListBegin()
-          for _i597 in xrange(_size593):
-            _elem598 = LockComponent()
-            _elem598.read(iprot)
-            self.component.append(_elem598)
+          (_etype610, _size607) = iprot.readListBegin()
+          for _i611 in xrange(_size607):
+            _elem612 = LockComponent()
+            _elem612.read(iprot)
+            self.component.append(_elem612)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13625,8 +13831,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter599 in self.component:
-        iter599.write(oprot)
+      for iter613 in self.component:
+        iter613.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -14324,11 +14530,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype603, _size600) = iprot.readListBegin()
-          for _i604 in xrange(_size600):
-            _elem605 = ShowLocksResponseElement()
-            _elem605.read(iprot)
-            self.locks.append(_elem605)
+          (_etype617, _size614) = iprot.readListBegin()
+          for _i618 in xrange(_size614):
+            _elem619 = ShowLocksResponseElement()
+            _elem619.read(iprot)
+            self.locks.append(_elem619)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14345,8 +14551,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter606 in self.locks:
-        iter606.write(oprot)
+      for iter620 in self.locks:
+        iter620.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14561,20 +14767,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype610, _size607) = iprot.readSetBegin()
-          for _i611 in xrange(_size607):
-            _elem612 = iprot.readI64()
-            self.aborted.add(_elem612)
+          (_etype624, _size621) = iprot.readSetBegin()
+          for _i625 in xrange(_size621):
+            _elem626 = iprot.readI64()
+            self.aborted.add(_elem626)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype616, _size613) = iprot.readSetBegin()
-          for _i617 in xrange(_size613):
-            _elem618 = iprot.readI64()
-            self.nosuch.add(_elem618)
+          (_etype630, _size627) = iprot.readSetBegin()
+          for _i631 in xrange(_size627):
+            _elem632 = iprot.readI64()
+            self.nosuch.add(_elem632)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -14591,15 +14797,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter619 in self.aborted:
-        oprot.writeI64(iter619)
+      for iter633 in self.aborted:
+        oprot.writeI64(iter633)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter620 in self.nosuch:
-        oprot.writeI64(iter620)
+      for iter634 in self.nosuch:
+        oprot.writeI64(iter634)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14696,11 +14902,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype622, _vtype623, _size621 ) = iprot.readMapBegin()
-          for _i625 in xrange(_size621):
-            _key626 = iprot.readString()
-            _val627 = iprot.readString()
-            self.properties[_key626] = _val627
+          (_ktype636, _vtype637, _size635 ) = iprot.readMapBegin()
+          for _i639 in xrange(_size635):
+            _key640 = iprot.readString()
+            _val641 = iprot.readString()
+            self.properties[_key640] = _val641
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -14737,9 +14943,9 @@ class CompactionRequest:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter628,viter629 in self.properties.items():
-        oprot.writeString(kiter628)
-        oprot.writeString(viter629)
+      for kiter642,viter643 in self.properties.items():
+        oprot.writeString(kiter642)
+        oprot.writeString(viter643)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15456,11 +15662,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype633, _size630) = iprot.readListBegin()
-          for _i634 in xrange(_size630):
-            _elem635 = ShowCompactResponseElement()
-            _elem635.read(iprot)
-            self.compacts.append(_elem635)
+          (_etype647, _size644) = iprot.readListBegin()
+          for _i648 in xrange(_size644):
+            _elem649 = ShowCompactResponseElement()
+            _elem649.read(iprot)
+            self.compacts.append(_elem649)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15477,8 +15683,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter636 in self.compacts:
-        iter636.write(oprot)
+      for iter650 in self.compacts:
+        iter650.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15567,10 +15773,10 @@ class AddDynamicPartitions:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype640, _size637) = iprot.readListBegin()
-          for _i641 in xrange(_size637):
-            _elem642 = iprot.readString()
-            self.partitionnames.append(_elem642)
+          (_etype654, _size651) = iprot.readListBegin()
+          for _i655 in xrange(_size651):
+            _elem656 = iprot.readString()
+            self.partitionnames.append(_elem656)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15608,8 +15814,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter643 in self.partitionnames:
-        oprot.writeString(iter643)
+      for iter657 in self.partitionnames:
+        oprot.writeString(iter657)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -15842,10 +16048,10 @@ class CreationMetadata:
       elif fid == 4:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype647, _size644) = iprot.readSetBegin()
-          for _i648 in xrange(_size644):
-            _elem649 = iprot.readString()
-            self.tablesUsed.add(_elem649)
+          (_etype661, _size658) = iprot.readSetBegin()
+          for _i662 in xrange(_size658):
+            _elem663 = iprot.readString()
+            self.tablesUsed.add(_elem663)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -15884,8 +16090,8 @@ class CreationMetadata:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 4)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter650 in self.tablesUsed:
-        oprot.writeString(iter650)
+      for iter664 in self.tablesUsed:
+        oprot.writeString(iter664)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -15974,10 +16180,10 @@ class NotificationEventRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.eventTypeSkipList = []
-          (_etype654, _size651) = iprot.readListBegin()
-          for _i655 in xrange(_size651):
-            _elem656 = iprot.readString()
-            self.eventTypeSkipList.append(_elem656)
+          (_etype668, _size665) = iprot.readListBegin()
+          for _i669 in xrange(_size665):
+            _elem670 = iprot.readString()
+            self.eventTypeSkipList.append(_elem670)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16002,8 +16208,8 @@ class NotificationEventRequest:
     if self.eventTypeSkipList is not None:
       oprot.writeFieldBegin('eventTypeSkipList', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.eventTypeSkipList))
-      for iter657 in self.eventTypeSkipList:
-        oprot.writeString(iter657)
+      for iter671 in self.eventTypeSkipList:
+        oprot.writeString(iter671)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16223,11 +16429,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype661, _size658) = iprot.readListBegin()
-          for _i662 in xrange(_size658):
-            _elem663 = NotificationEvent()
-            _elem663.read(iprot)
-            self.events.append(_elem663)
+          (_etype675, _size672) = iprot.readListBegin()
+          for _i676 in xrange(_size672):
+            _elem677 = NotificationEvent()
+            _elem677.read(iprot)
+            self.events.append(_elem677)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16244,8 +16450,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter664 in self.events:
-        iter664.write(oprot)
+      for iter678 in self.events:
+        iter678.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16568,30 +16774,30 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype668, _size665) = iprot.readListBegin()
-          for _i669 in xrange(_size665):
-            _elem670 = iprot.readString()
-            self.filesAdded.append(_elem670)
+          (_etype682, _size679) = iprot.readListBegin()
+          for _i683 in xrange(_size679):
+            _elem684 = iprot.readString()
+            self.filesAdded.append(_elem684)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype674, _size671) = iprot.readListBegin()
-          for _i675 in xrange(_size671):
-            _elem676 = iprot.readString()
-            self.filesAddedChecksum.append(_elem676)
+          (_etype688, _size685) = iprot.readListBegin()
+          for _i689 in xrange(_size685):
+            _elem690 = iprot.readString()
+            self.filesAddedChecksum.append(_elem690)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.subDirectoryList = []
-          (_etype680, _size677) = iprot.readListBegin()
-          for _i681 in xrange(_size677):
-            _elem682 = iprot.readString()
-            self.subDirectoryList.append(_elem682)
+          (_etype694, _size691) = iprot.readListBegin()
+          for _i695 in xrange(_size691):
+            _elem696 = iprot.readString()
+            self.subDirectoryList.append(_elem696)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16612,22 +16818,22 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter683 in self.filesAdded:
-        oprot.writeString(iter683)
+      for iter697 in self.filesAdded:
+        oprot.writeString(iter697)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter684 in self.filesAddedChecksum:
-        oprot.writeString(iter684)
+      for iter698 in self.filesAddedChecksum:
+        oprot.writeString(iter698)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.subDirectoryList is not None:
       oprot.writeFieldBegin('subDirectoryList', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.subDirectoryList))
-      for iter685 in self.subDirectoryList:
-        oprot.writeString(iter685)
+      for iter699 in self.subDirectoryList:
+        oprot.writeString(iter699)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16786,10 +16992,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype689, _size686) = iprot.readListBegin()
-          for _i690 in xrange(_size686):
-            _elem691 = iprot.readString()
-            self.partitionVals.append(_elem691)
+          (_etype703, _size700) = iprot.readListBegin()
+          for _i704 in xrange(_size700):
+            _elem705 = iprot.readString()
+            self.partitionVals.append(_elem705)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16827,8 +17033,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter692 in self.partitionVals:
-        oprot.writeString(iter692)
+      for iter706 in self.partitionVals:
+        oprot.writeString(iter706)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.catName is not None:
@@ -16980,10 +17186,10 @@ class WriteNotificationLogRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype696, _size693) = iprot.readListBegin()
-          for _i697 in xrange(_size693):
-            _elem698 = iprot.readString()
-            self.partitionVals.append(_elem698)
+          (_etype710, _size707) = iprot.readListBegin()
+          for _i711 in xrange(_size707):
+            _elem712 = iprot.readString()
+            self.partitionVals.append(_elem712)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17020,8 +17226,8 @@ class WriteNotificationLogRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter699 in self.partitionVals:
-        oprot.writeString(iter699)
+      for iter713 in self.partitionVals:
+        oprot.writeString(iter713)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17215,12 +17421,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype701, _vtype702, _size700 ) = iprot.readMapBegin()
-          for _i704 in xrange(_size700):
-            _key705 = iprot.readI64()
-            _val706 = MetadataPpdResult()
-            _val706.read(iprot)
-            self.metadata[_key705] = _val706
+          (_ktype715, _vtype716, _size714 ) = iprot.readMapBegin()
+          for _i718 in xrange(_size714):
+            _key719 = iprot.readI64()
+            _val720 = MetadataPpdResult()
+            _val720.read(iprot)
+            self.metadata[_key719] = _val720
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17242,9 +17448,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter707,viter708 in self.metadata.items():
-        oprot.writeI64(kiter707)
-        viter708.write(oprot)
+      for kiter721,viter722 in self.metadata.items():
+        oprot.writeI64(kiter721)
+        viter722.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -17314,10 +17520,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype712, _size709) = iprot.readListBegin()
-          for _i713 in xrange(_size709):
-            _elem714 = iprot.readI64()
-            self.fileIds.append(_elem714)
+          (_etype726, _size723) = iprot.readListBegin()
+          for _i727 in xrange(_size723):
+            _elem728 = iprot.readI64()
+            self.fileIds.append(_elem728)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17349,8 +17555,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter715 in self.fileIds:
-        oprot.writeI64(iter715)
+      for iter729 in self.fileIds:
+        oprot.writeI64(iter729)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -17424,11 +17630,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype717, _vtype718, _size716 ) = iprot.readMapBegin()
-          for _i720 in xrange(_size716):
-            _key721 = iprot.readI64()
-            _val722 = iprot.readString()
-            self.metadata[_key721] = _val722
+          (_ktype731, _vtype732, _size730 ) = iprot.readMapBegin()
+          for _i734 in xrange(_size730):
+            _key735 = iprot.readI64()
+            _val736 = iprot.readString()
+            self.metadata[_key735] = _val736
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17450,9 +17656,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter723,viter724 in self.metadata.items():
-        oprot.writeI64(kiter723)
-        oprot.writeString(viter724)
+      for kiter737,viter738 in self.metadata.items():
+        oprot.writeI64(kiter737)
+        oprot.writeString(viter738)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -17513,10 +17719,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype728, _size725) = iprot.readListBegin()
-          for _i729 in xrange(_size725):
-            _elem730 = iprot.readI64()
-            self.fileIds.append(_elem730)
+          (_etype742, _size739) = iprot.readListBegin()
+          for _i743 in xrange(_size739):
+            _elem744 = iprot.readI64()
+            self.fileIds.append(_elem744)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17533,8 +17739,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter731 in self.fileIds:
-        oprot.writeI64(iter731)
+      for iter745 in self.fileIds:
+        oprot.writeI64(iter745)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17640,20 +17846,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype735, _size732) = iprot.readListBegin()
-          for _i736 in xrange(_size732):
-            _elem737 = iprot.readI64()
-            self.fileIds.append(_elem737)
+          (_etype749, _size746) = iprot.readListBegin()
+          for _i750 in xrange(_size746):
+            _elem751 = iprot.readI64()
+            self.fileIds.append(_elem751)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = iprot.readString()
-            self.metadata.append(_elem743)
+          (_etype755, _size752) = iprot.readListBegin()
+          for _i756 in xrange(_size752):
+            _elem757 = iprot.readString()
+            self.metadata.append(_elem757)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17675,15 +17881,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter744 in self.fileIds:
-        oprot.writeI64(iter744)
+      for iter758 in self.fileIds:
+        oprot.writeI64(iter758)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter745 in self.metadata:
-        oprot.writeString(iter745)
+      for iter759 in self.metadata:
+        oprot.writeString(iter759)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -17791,10 +17997,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype749, _size746) = iprot.readListBegin()
-          for _i750 in xrange(_size746):
-            _elem751 = iprot.readI64()
-            self.fileIds.append(_elem751)
+          (_etype763, _size760) = iprot.readListBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = iprot.readI64()
+            self.fileIds.append(_elem765)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17811,8 +18017,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter752 in self.fileIds:
-        oprot.writeI64(iter752)
+      for iter766 in self.fileIds:
+        oprot.writeI64(iter766)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18041,11 +18247,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype756, _size753) = iprot.readListBegin()
-          for _i757 in xrange(_size753):
-            _elem758 = Function()
-            _elem758.read(iprot)
-            self.functions.append(_elem758)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = Function()
+            _elem772.read(iprot)
+            self.functions.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18062,8 +18268,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter759 in self.functions:
-        iter759.write(oprot)
+      for iter773 in self.functions:
+        iter773.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18115,10 +18321,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readI32()
-            self.values.append(_elem765)
+          (_etype777, _size774) = iprot.readListBegin()
+          for _i778 in xrange(_size774):
+            _elem779 = iprot.readI32()
+            self.values.append(_elem779)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18135,8 +18341,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter766 in self.values:
-        oprot.writeI32(iter766)
+      for iter780 in self.values:
+        oprot.writeI32(iter780)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18421,10 +18627,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype770, _size767) = iprot.readListBegin()
-          for _i771 in xrange(_size767):
-            _elem772 = iprot.readString()
-            self.tblNames.append(_elem772)
+          (_etype784, _size781) = iprot.readListBegin()
+          for _i785 in xrange(_size781):
+            _elem786 = iprot.readString()
+            self.tblNames.append(_elem786)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18456,8 +18662,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter773 in self.tblNames:
-        oprot.writeString(iter773)
+      for iter787 in self.tblNames:
+        oprot.writeString(iter787)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -18522,11 +18728,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype777, _size774) = iprot.readListBegin()
-          for _i778 in xrange(_size774):
-            _elem779 = Table()
-            _elem779.read(iprot)
-            self.tables.append(_elem779)
+          (_etype791, _size788) = iprot.readListBegin()
+          for _i792 in xrange(_size788):
+            _elem793 = Table()
+            _elem793.read(iprot)
+            self.tables.append(_elem793)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18543,8 +18749,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter780 in self.tables:
-        iter780.write(oprot)
+      for iter794 in self.tables:
+        iter794.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19843,44 +20049,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype784, _size781) = iprot.readListBegin()
-          for _i785 in xrange(_size781):
-            _elem786 = WMPool()
-            _elem786.read(iprot)
-            self.pools.append(_elem786)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = WMPool()
+            _elem800.read(iprot)
+            self.pools.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = WMMapping()
-            _elem792.read(iprot)
-            self.mappings.append(_elem792)
+          (_etype804, _size801) = iprot.readListBegin()
+          for _i805 in xrange(_size801):
+            _elem806 = WMMapping()
+            _elem806.read(iprot)
+            self.mappings.append(_elem806)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype796, _size793) = iprot.readListBegin()
-          for _i797 in xrange(_size793):
-            _elem798 = WMTrigger()
-            _elem798.read(iprot)
-            self.triggers.append(_elem798)
+          (_etype810, _size807) = iprot.readListBegin()
+          for _i811 in xrange(_size807):
+            _elem812 = WMTrigger()
+            _elem812.read(iprot)
+            self.triggers.append(_elem812)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype802, _size799) = iprot.readListBegin()
-          for _i803 in xrange(_size799):
-            _elem804 = WMPoolTrigger()
-            _elem804.read(iprot)
-            self.poolTriggers.append(_elem804)
+          (_etype816, _size813) = iprot.readListBegin()
+          for _i817 in xrange(_size813):
+            _elem818 = WMPoolTrigger()
+            _elem818.read(iprot)
+            self.poolTriggers.append(_elem818)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19901,29 +20107,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter805 in self.pools:
-        iter805.write(oprot)
+      for iter819 in self.pools:
+        iter819.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter806 in self.mappings:
-        iter806.write(oprot)
+      for iter820 in self.mappings:
+        iter820.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter807 in self.triggers:
-        iter807.write(oprot)
+      for iter821 in self.triggers:
+        iter821.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter808 in self.poolTriggers:
-        iter808.write(oprot)
+      for iter822 in self.poolTriggers:
+        iter822.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20448,11 +20654,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype812, _size809) = iprot.readListBegin()
-          for _i813 in xrange(_size809):
-            _elem814 = WMResourcePlan()
-            _elem814.read(iprot)
-            self.resourcePlans.append(_elem814)
+          (_etype826, _size823) = iprot.readListBegin()
+          for _i827 in xrange(_size823):
+            _elem828 = WMResourcePlan()
+            _elem828.read(iprot)
+            self.resourcePlans.append(_elem828)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20469,8 +20675,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter815 in self.resourcePlans:
-        iter815.write(oprot)
+      for iter829 in self.resourcePlans:
+        iter829.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20800,20 +21006,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype819, _size816) = iprot.readListBegin()
-          for _i820 in xrange(_size816):
-            _elem821 = iprot.readString()
-            self.errors.append(_elem821)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = iprot.readString()
+            self.errors.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = iprot.readString()
-            self.warnings.append(_elem827)
+          (_etype839, _size836) = iprot.readListBegin()
+          for _i840 in xrange(_size836):
+            _elem841 = iprot.readString()
+            self.warnings.append(_elem841)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20830,15 +21036,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter828 in self.errors:
-        oprot.writeString(iter828)
+      for iter842 in self.errors:
+        oprot.writeString(iter842)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter829 in self.warnings:
-        oprot.writeString(iter829)
+      for iter843 in self.warnings:
+        oprot.writeString(iter843)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21454,11 +21660,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype833, _size830) = iprot.readListBegin()
-          for _i834 in xrange(_size830):
-            _elem835 = WMTrigger()
-            _elem835.read(iprot)
-            self.triggers.append(_elem835)
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = WMTrigger()
+            _elem849.read(iprot)
+            self.triggers.append(_elem849)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21475,8 +21681,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter836 in self.triggers:
-        iter836.write(oprot)
+      for iter850 in self.triggers:
+        iter850.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22686,11 +22892,11 @@ class SchemaVersion:
       elif fid == 4:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype840, _size837) = iprot.readListBegin()
-          for _i841 in xrange(_size837):
-            _elem842 = FieldSchema()
-            _elem842.read(iprot)
-            self.cols.append(_elem842)
+          (_etype854, _size851) = iprot.readListBegin()
+          for _i855 in xrange(_size851):
+            _elem856 = FieldSchema()
+            _elem856.read(iprot)
+            self.cols.append(_elem856)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22750,8 +22956,8 @@ class SchemaVersion:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter843 in self.cols:
-        iter843.write(oprot)
+      for iter857 in self.cols:
+        iter857.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.state is not None:
@@ -23006,11 +23212,11 @@ class FindSchemasByColsResp:
       if fid == 1:
         if ftype == TType.LIST:
           self.schemaVersions = []
-          (_etype847, _size844) = iprot.readListBegin()
-          for _i848 in xrange(_size844):
-            _elem849 = SchemaVersionDescriptor()
-            _elem849.read(iprot)
-            self.schemaVersions.append(_elem849)
+          (_etype861, _size858) = iprot.readListBegin()
+          for _i862 in xrange(_size858):
+            _elem863 = SchemaVersionDescriptor()
+            _elem863.read(iprot)
+            self.schemaVersions.append(_elem863)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23027,8 +23233,8 @@ class FindSchemasByColsResp:
     if self.schemaVersions is not None:
       oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
-      for iter850 in self.schemaVersions:
-        iter850.write(oprot)
+      for iter864 in self.schemaVersions:
+        iter864.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23513,11 +23719,11 @@ class AlterPartitionsRequest:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitions = []
-          (_etype854, _size851) = iprot.readListBegin()
-          for _i855 in xrange(_size851):
-            _elem856 = Partition()
-            _elem856.read(iprot)
-            self.partitions.append(_elem856)
+          (_etype868, _size865) = iprot.readListBegin()
+          for _i869 in xrange(_size865):
+            _elem870 = Partition()
+            _elem870.read(iprot)
+            self.partitions.append(_elem870)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23562,8 +23768,8 @@ class AlterPartitionsRequest:
     if self.partitions is not None:
       oprot.writeFieldBegin('partitions', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-      for iter857 in self.partitions:
-        iter857.write(oprot)
+      for iter871 in self.partitions:
+        iter871.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environmentContext is not None:
@@ -23715,10 +23921,10 @@ class RenamePartitionRequest:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partVals = []
-          (_etype861, _size858) = iprot.readListBegin()
-          for _i862 in xrange(_size858):
-            _elem863 = iprot.readString()
-            self.partVals.append(_elem863)
+          (_etype875, _size872) = iprot.readListBegin()
+          for _i876 in xrange(_size872):
+            _elem877 = iprot.readString()
+            self.partVals.append(_elem877)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23758,8 +23964,8 @@ class RenamePartitionRequest:
     if self.partVals is not None:
       oprot.writeFieldBegin('partVals', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partVals))
-      for iter864 in self.partVals:
-        oprot.writeString(iter864)
+      for iter878 in self.partVals:
+        oprot.writeString(iter878)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.newPart is not None:
@@ -24081,10 +24287,10 @@ class GetPartitionsProjectionSpec:
       if fid == 1:
         if ftype == TType.LIST:
           self.fieldList = []
-          (_etype868, _size865) = iprot.readListBegin()
-          for _i869 in xrange(_size865):
-            _elem870 = iprot.readString()
-            self.fieldList.append(_elem870)
+          (_etype882, _size879) = iprot.readListBegin()
+          for _i883 in xrange(_size879):
+            _elem884 = iprot.readString()
+            self.fieldList.append(_elem884)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24111,8 +24317,8 @@ class GetPartitionsProjectionSpec:
     if self.fieldList is not None:
       oprot.writeFieldBegin('fieldList', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.fieldList))
-      for iter871 in self.fieldList:
-        oprot.writeString(iter871)
+      for iter885 in self.fieldList:
+        oprot.writeString(iter885)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.includeParamKeyPattern is not None:
@@ -24188,10 +24394,10 @@ class GetPartitionsFilterSpec:
       elif fid == 8:
         if ftype == TType.LIST:
           self.filters = []
-          (_etype875, _size872) = iprot.readListBegin()
-          for _i876 in xrange(_size872):
-            _elem877 = iprot.readString()
-            self.filters.append(_elem877)
+          (_etype889, _size886) = iprot.readListBegin()
+          for _i890 in xrange(_size886):
+            _elem891 = iprot.readString()
+            self.filters.append(_elem891)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24212,8 +24418,8 @@ class GetPartitionsFilterSpec:
     if self.filters is not None:
       oprot.writeFieldBegin('filters', TType.LIST, 8)
       oprot.writeListBegin(TType.STRING, len(self.filters))
-      for iter878 in self.filters:
-        oprot.writeString(iter878)
+      for iter892 in self.filters:
+        oprot.writeString(iter892)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24266,11 +24472,11 @@ class GetPartitionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.partitionSpec = []
-          (_etype882, _size879) = iprot.readListBegin()
-          for _i883 in xrange(_size879):
-            _elem884 = PartitionSpec()
-            _elem884.read(iprot)
-            self.partitionSpec.append(_elem884)
+          (_etype896, _size893) = iprot.readListBegin()
+          for _i897 in xrange(_size893):
+            _elem898 = PartitionSpec()
+            _elem898.read(iprot)
+            self.partitionSpec.append(_elem898)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24287,8 +24493,8 @@ class GetPartitionsResponse:
     if self.partitionSpec is not None:
       oprot.writeFieldBegin('partitionSpec', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.partitionSpec))
-      for iter885 in self.partitionSpec:
-        iter885.write(oprot)
+      for iter899 in self.partitionSpec:
+        iter899.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24386,10 +24592,10 @@ class GetPartitionsRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.groupNames = []
-          (_etype889, _size886) = iprot.readListBegin()
-          for _i890 in xrange(_size886):
-            _elem891 = iprot.readString()
-            self.groupNames.append(_elem891)
+          (_etype903, _size900) = iprot.readListBegin()
+          for _i904 in xrange(_size900):
+            _elem905 = iprot.readString()
+            self.groupNames.append(_elem905)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24438,8 +24644,8 @@ class GetPartitionsRequest:
     if self.groupNames is not None:
       oprot.writeFieldBegin('groupNames', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.groupNames))
-      for iter892 in self.groupNames:
-        oprot.writeString(iter892)
+      for iter906 in self.groupNames:
+        oprot.writeString(iter906)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.projectionSpec is not None:

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index e885194..ac95307 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -1178,6 +1178,7 @@ class Partition
   CATNAME = 9
   WRITEID = 10
   ISSTATSCOMPLIANT = 11
+  COLSTATS = 12
 
   FIELDS = {
     VALUES => {:type => ::Thrift::Types::LIST, :name => 'values', :element => {:type => ::Thrift::Types::STRING}},
@@ -1190,7 +1191,8 @@ class Partition
     PRIVILEGES => {:type => ::Thrift::Types::STRUCT, :name => 'privileges', :class => ::PrincipalPrivilegeSet, :optional => true},
     CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true},
     WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId', :default => -1, :optional => true},
-    ISSTATSCOMPLIANT => {:type => ::Thrift::Types::BOOL, :name => 'isStatsCompliant', :optional => true}
+    ISSTATSCOMPLIANT => {:type => ::Thrift::Types::BOOL, :name => 'isStatsCompliant', :optional => true},
+    COLSTATS => {:type => ::Thrift::Types::STRUCT, :name => 'colStats', :class => ::ColumnStatistics, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2468,6 +2470,47 @@ class PartitionValuesResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class GetPartitionsByNamesRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DB_NAME = 1
+  TBL_NAME = 2
+  NAMES = 3
+  GET_COL_STATS = 4
+
+  FIELDS = {
+    DB_NAME => {:type => ::Thrift::Types::STRING, :name => 'db_name'},
+    TBL_NAME => {:type => ::Thrift::Types::STRING, :name => 'tbl_name'},
+    NAMES => {:type => ::Thrift::Types::LIST, :name => 'names', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    GET_COL_STATS => {:type => ::Thrift::Types::BOOL, :name => 'get_col_stats', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db_name is unset!') unless @db_name
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tbl_name is unset!') unless @tbl_name
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetPartitionsByNamesResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  PARTITIONS = 1
+
+  FIELDS = {
+    PARTITIONS => {:type => ::Thrift::Types::LIST, :name => 'partitions', :element => {:type => ::Thrift::Types::STRUCT, :class => ::Partition}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field partitions is unset!') unless @partitions
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class ResourceUri
   include ::Thrift::Struct, ::Thrift::Struct_Union
   RESOURCETYPE = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 35e1693..39c671a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -1417,6 +1417,23 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_partitions_by_names failed: unknown result')
     end
 
+    def get_partitions_by_names_req(req)
+      send_get_partitions_by_names_req(req)
+      return recv_get_partitions_by_names_req()
+    end
+
+    def send_get_partitions_by_names_req(req)
+      send_message('get_partitions_by_names_req', Get_partitions_by_names_req_args, :req => req)
+    end
+
+    def recv_get_partitions_by_names_req()
+      result = receive_message(Get_partitions_by_names_req_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_partitions_by_names_req failed: unknown result')
+    end
+
     def alter_partition(db_name, tbl_name, new_part)
       send_alter_partition(db_name, tbl_name, new_part)
       recv_alter_partition()
@@ -4814,6 +4831,19 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_partitions_by_names', seqid)
     end
 
+    def process_get_partitions_by_names_req(seqid, iprot, oprot)
+      args = read_args(iprot, Get_partitions_by_names_req_args)
+      result = Get_partitions_by_names_req_result.new()
+      begin
+        result.success = @handler.get_partitions_by_names_req(args.req)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_partitions_by_names_req', seqid)
+    end
+
     def process_alter_partition(seqid, iprot, oprot)
       args = read_args(iprot, Alter_partition_args)
       result = Alter_partition_result.new()
@@ -9671,6 +9701,42 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_partitions_by_names_req_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::GetPartitionsByNamesRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_partitions_by_names_req_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetPartitionsByNamesResult},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Alter_partition_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     DB_NAME = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 59b5bcf..a1826fa 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -1831,11 +1831,28 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
+  public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
+                                              List<String> part_names, boolean getColStats)
+          throws TException {
+    return getPartitionsByNames(getDefaultCatalog(conf), db_name, tbl_name, part_names, getColStats);
+  }
+
+  @Override
   public List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
                                               List<String> part_names) throws TException {
+    return getPartitionsByNames(catName, db_name, tbl_name, part_names, false);
+  }
+
+  @Override
+  public List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
+                                              List<String> part_names, boolean getColStats) throws TException {
     checkDbAndTableFilters(catName, db_name, tbl_name);
-    List<Partition> parts =
-        client.get_partitions_by_names(prependCatalogToDbName(catName, db_name, conf), tbl_name, part_names);
+    GetPartitionsByNamesRequest gpbnr =
+            new GetPartitionsByNamesRequest(prependCatalogToDbName(catName, db_name, conf),
+                    tbl_name);
+    gpbnr.setNames(part_names);
+    gpbnr.setGet_col_stats(getColStats);
+    List<Partition> parts = client.get_partitions_by_names_req(gpbnr).getPartitions();
     return deepCopyPartitions(FilterUtils.filterPartitionsIfEnabled(isClientFilterEnabled, filterHook, parts));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index e10cc8c..f67761e 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1478,6 +1478,20 @@ public interface IMetaStoreClient {
 
   /**
    * Get partitions by a list of partition names.
+   * @param db_name database name
+   * @param tbl_name table name
+   * @param part_names list of partition names
+   * @param getColStats if true include statistics in the Partition object
+   * @return list of Partition objects
+   * @throws NoSuchObjectException No such partitions
+   * @throws MetaException error accessing the RDBMS.
+   * @throws TException thrift transport error
+   */
+  List<Partition> getPartitionsByNames(String db_name, String tbl_name, List<String> part_names,
+      boolean getColStats) throws NoSuchObjectException, MetaException, TException;
+
+  /**
+   * Get partitions by a list of partition names.
    * @param catName catalog name
    * @param db_name database name
    * @param tbl_name table name
@@ -1491,6 +1505,22 @@ public interface IMetaStoreClient {
                                        List<String> part_names)
       throws NoSuchObjectException, MetaException, TException;
 
+    /**
+     * Get partitions by a list of partition names.
+     * @param catName catalog name
+     * @param db_name database name
+     * @param tbl_name table name
+     * @param part_names list of partition names
+     * @param getColStats if true, column statistics is added to the Partition objects
+     * @return list of Partition objects
+     * @throws NoSuchObjectException No such partitions
+     * @throws MetaException error accessing the RDBMS.
+     * @throws TException thrift transport error
+     */
+    List<Partition> getPartitionsByNames(String catName, String db_name, String tbl_name,
+                                         List<String> part_names, boolean getColStats)
+            throws NoSuchObjectException, MetaException, TException;
+
   /**
    * List partitions along with privilege information for a user or groups
    * @param dbName database name

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index c58015d..19bc713 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -462,7 +462,8 @@ struct Partition {
   8: optional PrincipalPrivilegeSet privileges,
   9: optional string catName,
   10: optional i64 writeId=-1,
-  11: optional bool isStatsCompliant
+  11: optional bool isStatsCompliant,
+  12: optional ColumnStatistics colStats // column statistics for partition
 }
 
 struct PartitionWithoutSD {
@@ -826,6 +827,17 @@ struct PartitionValuesResponse {
   1: required list<PartitionValuesRow> partitionValues;
 }
 
+struct GetPartitionsByNamesRequest {
+  1: required string db_name,
+  2: required string tbl_name,
+  3: optional list<string> names,
+  4: optional bool get_col_stats
+}
+
+struct GetPartitionsByNamesResult {
+  1: required list<Partition> partitions
+}
+
 enum FunctionType {
   JAVA = 1,
 }
@@ -2103,6 +2115,8 @@ service ThriftHiveMetastore extends fb303.FacebookService
   // get partitions give a list of partition names
   list<Partition> get_partitions_by_names(1:string db_name 2:string tbl_name 3:list<string> names)
                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
+  GetPartitionsByNamesResult get_partitions_by_names_req(1:GetPartitionsByNamesRequest req)
+                        throws(1:MetaException o1, 2:NoSuchObjectException o2)
 
   // changes the partition to the new partition object. partition is identified from the part values
   // in the new_part


[06/11] hive git commit: HIVE-21079: Replicate column statistics for partitions of partitioned table (Ashutosh Bapat, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index f92e23e..4b38aeb 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list888.size);
-                WMPool _elem889;
-                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list904.size);
+                WMPool _elem905;
+                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
                 {
-                  _elem889 = new WMPool();
-                  _elem889.read(iprot);
-                  struct.pools.add(_elem889);
+                  _elem905 = new WMPool();
+                  _elem905.read(iprot);
+                  struct.pools.add(_elem905);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list891 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list891.size);
-                WMMapping _elem892;
-                for (int _i893 = 0; _i893 < _list891.size; ++_i893)
+                org.apache.thrift.protocol.TList _list907 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list907.size);
+                WMMapping _elem908;
+                for (int _i909 = 0; _i909 < _list907.size; ++_i909)
                 {
-                  _elem892 = new WMMapping();
-                  _elem892.read(iprot);
-                  struct.mappings.add(_elem892);
+                  _elem908 = new WMMapping();
+                  _elem908.read(iprot);
+                  struct.mappings.add(_elem908);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list894 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list894.size);
-                WMTrigger _elem895;
-                for (int _i896 = 0; _i896 < _list894.size; ++_i896)
+                org.apache.thrift.protocol.TList _list910 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list910.size);
+                WMTrigger _elem911;
+                for (int _i912 = 0; _i912 < _list910.size; ++_i912)
                 {
-                  _elem895 = new WMTrigger();
-                  _elem895.read(iprot);
-                  struct.triggers.add(_elem895);
+                  _elem911 = new WMTrigger();
+                  _elem911.read(iprot);
+                  struct.triggers.add(_elem911);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list897 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list897.size);
-                WMPoolTrigger _elem898;
-                for (int _i899 = 0; _i899 < _list897.size; ++_i899)
+                org.apache.thrift.protocol.TList _list913 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list913.size);
+                WMPoolTrigger _elem914;
+                for (int _i915 = 0; _i915 < _list913.size; ++_i915)
                 {
-                  _elem898 = new WMPoolTrigger();
-                  _elem898.read(iprot);
-                  struct.poolTriggers.add(_elem898);
+                  _elem914 = new WMPoolTrigger();
+                  _elem914.read(iprot);
+                  struct.poolTriggers.add(_elem914);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter900 : struct.pools)
+          for (WMPool _iter916 : struct.pools)
           {
-            _iter900.write(oprot);
+            _iter916.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter901 : struct.mappings)
+            for (WMMapping _iter917 : struct.mappings)
             {
-              _iter901.write(oprot);
+              _iter917.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter902 : struct.triggers)
+            for (WMTrigger _iter918 : struct.triggers)
             {
-              _iter902.write(oprot);
+              _iter918.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter903 : struct.poolTriggers)
+            for (WMPoolTrigger _iter919 : struct.poolTriggers)
             {
-              _iter903.write(oprot);
+              _iter919.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter904 : struct.pools)
+        for (WMPool _iter920 : struct.pools)
         {
-          _iter904.write(oprot);
+          _iter920.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter905 : struct.mappings)
+          for (WMMapping _iter921 : struct.mappings)
           {
-            _iter905.write(oprot);
+            _iter921.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter906 : struct.triggers)
+          for (WMTrigger _iter922 : struct.triggers)
           {
-            _iter906.write(oprot);
+            _iter922.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter907 : struct.poolTriggers)
+          for (WMPoolTrigger _iter923 : struct.poolTriggers)
           {
-            _iter907.write(oprot);
+            _iter923.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list908 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list908.size);
-        WMPool _elem909;
-        for (int _i910 = 0; _i910 < _list908.size; ++_i910)
+        org.apache.thrift.protocol.TList _list924 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list924.size);
+        WMPool _elem925;
+        for (int _i926 = 0; _i926 < _list924.size; ++_i926)
         {
-          _elem909 = new WMPool();
-          _elem909.read(iprot);
-          struct.pools.add(_elem909);
+          _elem925 = new WMPool();
+          _elem925.read(iprot);
+          struct.pools.add(_elem925);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list911 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list911.size);
-          WMMapping _elem912;
-          for (int _i913 = 0; _i913 < _list911.size; ++_i913)
+          org.apache.thrift.protocol.TList _list927 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list927.size);
+          WMMapping _elem928;
+          for (int _i929 = 0; _i929 < _list927.size; ++_i929)
           {
-            _elem912 = new WMMapping();
-            _elem912.read(iprot);
-            struct.mappings.add(_elem912);
+            _elem928 = new WMMapping();
+            _elem928.read(iprot);
+            struct.mappings.add(_elem928);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list914 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list914.size);
-          WMTrigger _elem915;
-          for (int _i916 = 0; _i916 < _list914.size; ++_i916)
+          org.apache.thrift.protocol.TList _list930 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list930.size);
+          WMTrigger _elem931;
+          for (int _i932 = 0; _i932 < _list930.size; ++_i932)
           {
-            _elem915 = new WMTrigger();
-            _elem915.read(iprot);
-            struct.triggers.add(_elem915);
+            _elem931 = new WMTrigger();
+            _elem931.read(iprot);
+            struct.triggers.add(_elem931);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list917.size);
-          WMPoolTrigger _elem918;
-          for (int _i919 = 0; _i919 < _list917.size; ++_i919)
+          org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list933.size);
+          WMPoolTrigger _elem934;
+          for (int _i935 = 0; _i935 < _list933.size; ++_i935)
           {
-            _elem918 = new WMPoolTrigger();
-            _elem918.read(iprot);
-            struct.poolTriggers.add(_elem918);
+            _elem934 = new WMPoolTrigger();
+            _elem934.read(iprot);
+            struct.poolTriggers.add(_elem934);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index cd20b15..e97eab3 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list920.size);
-                WMResourcePlan _elem921;
-                for (int _i922 = 0; _i922 < _list920.size; ++_i922)
+                org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list936.size);
+                WMResourcePlan _elem937;
+                for (int _i938 = 0; _i938 < _list936.size; ++_i938)
                 {
-                  _elem921 = new WMResourcePlan();
-                  _elem921.read(iprot);
-                  struct.resourcePlans.add(_elem921);
+                  _elem937 = new WMResourcePlan();
+                  _elem937.read(iprot);
+                  struct.resourcePlans.add(_elem937);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter923 : struct.resourcePlans)
+            for (WMResourcePlan _iter939 : struct.resourcePlans)
             {
-              _iter923.write(oprot);
+              _iter939.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter924 : struct.resourcePlans)
+          for (WMResourcePlan _iter940 : struct.resourcePlans)
           {
-            _iter924.write(oprot);
+            _iter940.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list925.size);
-          WMResourcePlan _elem926;
-          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list941.size);
+          WMResourcePlan _elem942;
+          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
           {
-            _elem926 = new WMResourcePlan();
-            _elem926.read(iprot);
-            struct.resourcePlans.add(_elem926);
+            _elem942 = new WMResourcePlan();
+            _elem942.read(iprot);
+            struct.resourcePlans.add(_elem942);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 0fc76b9..4a6633b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list944.size);
-                WMTrigger _elem945;
-                for (int _i946 = 0; _i946 < _list944.size; ++_i946)
+                org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list960.size);
+                WMTrigger _elem961;
+                for (int _i962 = 0; _i962 < _list960.size; ++_i962)
                 {
-                  _elem945 = new WMTrigger();
-                  _elem945.read(iprot);
-                  struct.triggers.add(_elem945);
+                  _elem961 = new WMTrigger();
+                  _elem961.read(iprot);
+                  struct.triggers.add(_elem961);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter947 : struct.triggers)
+            for (WMTrigger _iter963 : struct.triggers)
             {
-              _iter947.write(oprot);
+              _iter963.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter948 : struct.triggers)
+          for (WMTrigger _iter964 : struct.triggers)
           {
-            _iter948.write(oprot);
+            _iter964.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list949.size);
-          WMTrigger _elem950;
-          for (int _i951 = 0; _i951 < _list949.size; ++_i951)
+          org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list965.size);
+          WMTrigger _elem966;
+          for (int _i967 = 0; _i967 < _list965.size; ++_i967)
           {
-            _elem950 = new WMTrigger();
-            _elem950.read(iprot);
-            struct.triggers.add(_elem950);
+            _elem966 = new WMTrigger();
+            _elem966.read(iprot);
+            struct.triggers.add(_elem966);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index deb1569..b01a517 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list928.size);
-                String _elem929;
-                for (int _i930 = 0; _i930 < _list928.size; ++_i930)
+                org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list944.size);
+                String _elem945;
+                for (int _i946 = 0; _i946 < _list944.size; ++_i946)
                 {
-                  _elem929 = iprot.readString();
-                  struct.errors.add(_elem929);
+                  _elem945 = iprot.readString();
+                  struct.errors.add(_elem945);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list931 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list931.size);
-                String _elem932;
-                for (int _i933 = 0; _i933 < _list931.size; ++_i933)
+                org.apache.thrift.protocol.TList _list947 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list947.size);
+                String _elem948;
+                for (int _i949 = 0; _i949 < _list947.size; ++_i949)
                 {
-                  _elem932 = iprot.readString();
-                  struct.warnings.add(_elem932);
+                  _elem948 = iprot.readString();
+                  struct.warnings.add(_elem948);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter934 : struct.errors)
+            for (String _iter950 : struct.errors)
             {
-              oprot.writeString(_iter934);
+              oprot.writeString(_iter950);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter935 : struct.warnings)
+            for (String _iter951 : struct.warnings)
             {
-              oprot.writeString(_iter935);
+              oprot.writeString(_iter951);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter936 : struct.errors)
+          for (String _iter952 : struct.errors)
           {
-            oprot.writeString(_iter936);
+            oprot.writeString(_iter952);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter937 : struct.warnings)
+          for (String _iter953 : struct.warnings)
           {
-            oprot.writeString(_iter937);
+            oprot.writeString(_iter953);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list938 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list938.size);
-          String _elem939;
-          for (int _i940 = 0; _i940 < _list938.size; ++_i940)
+          org.apache.thrift.protocol.TList _list954 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list954.size);
+          String _elem955;
+          for (int _i956 = 0; _i956 < _list954.size; ++_i956)
           {
-            _elem939 = iprot.readString();
-            struct.errors.add(_elem939);
+            _elem955 = iprot.readString();
+            struct.errors.add(_elem955);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list941.size);
-          String _elem942;
-          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
+          org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list957.size);
+          String _elem958;
+          for (int _i959 = 0; _i959 < _list957.size; ++_i959)
           {
-            _elem942 = iprot.readString();
-            struct.warnings.add(_elem942);
+            _elem958 = iprot.readString();
+            struct.warnings.add(_elem958);
           }
         }
         struct.setWarningsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/71dfd1d1/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
index 57f50b7..500293c 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
@@ -813,13 +813,13 @@ import org.slf4j.LoggerFactory;
           case 6: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list788.size);
-                String _elem789;
-                for (int _i790 = 0; _i790 < _list788.size; ++_i790)
+                org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list804.size);
+                String _elem805;
+                for (int _i806 = 0; _i806 < _list804.size; ++_i806)
                 {
-                  _elem789 = iprot.readString();
-                  struct.partitionVals.add(_elem789);
+                  _elem805 = iprot.readString();
+                  struct.partitionVals.add(_elem805);
                 }
                 iprot.readListEnd();
               }
@@ -867,9 +867,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter791 : struct.partitionVals)
+            for (String _iter807 : struct.partitionVals)
             {
-              oprot.writeString(_iter791);
+              oprot.writeString(_iter807);
             }
             oprot.writeListEnd();
           }
@@ -906,9 +906,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter792 : struct.partitionVals)
+          for (String _iter808 : struct.partitionVals)
           {
-            oprot.writeString(_iter792);
+            oprot.writeString(_iter808);
           }
         }
       }
@@ -931,13 +931,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list793.size);
-          String _elem794;
-          for (int _i795 = 0; _i795 < _list793.size; ++_i795)
+          org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list809.size);
+          String _elem810;
+          for (int _i811 = 0; _i811 < _list809.size; ++_i811)
           {
-            _elem794 = iprot.readString();
-            struct.partitionVals.add(_elem794);
+            _elem810 = iprot.readString();
+            struct.partitionVals.add(_elem810);
           }
         }
         struct.setPartitionValsIsSet(true);