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

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

Repository: hive
Updated Branches:
  refs/heads/master 793681c76 -> c57a59611


http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 4db9680..8d88cd7 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -9984,6 +9984,7 @@ class CompactionRequest:
    - partitionname
    - type
    - runas
+   - properties
   """
 
   thrift_spec = (
@@ -9993,14 +9994,16 @@ class CompactionRequest:
     (3, TType.STRING, 'partitionname', None, None, ), # 3
     (4, TType.I32, 'type', None, None, ), # 4
     (5, TType.STRING, 'runas', None, None, ), # 5
+    (6, TType.MAP, 'properties', (TType.STRING,None,TType.STRING,None), None, ), # 6
   )
 
-  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, runas=None,):
+  def __init__(self, dbname=None, tablename=None, partitionname=None, type=None, runas=None, properties=None,):
     self.dbname = dbname
     self.tablename = tablename
     self.partitionname = partitionname
     self.type = type
     self.runas = runas
+    self.properties = properties
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -10036,6 +10039,17 @@ class CompactionRequest:
           self.runas = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.MAP:
+          self.properties = {}
+          (_ktype463, _vtype464, _size462 ) = iprot.readMapBegin()
+          for _i466 in xrange(_size462):
+            _key467 = iprot.readString()
+            _val468 = iprot.readString()
+            self.properties[_key467] = _val468
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -10066,6 +10080,14 @@ class CompactionRequest:
       oprot.writeFieldBegin('runas', TType.STRING, 5)
       oprot.writeString(self.runas)
       oprot.writeFieldEnd()
+    if self.properties is not None:
+      oprot.writeFieldBegin('properties', TType.MAP, 6)
+      oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
+      for kiter469,viter470 in self.properties.items():
+        oprot.writeString(kiter469)
+        oprot.writeString(viter470)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -10086,6 +10108,7 @@ class CompactionRequest:
     value = (value * 31) ^ hash(self.partitionname)
     value = (value * 31) ^ hash(self.type)
     value = (value * 31) ^ hash(self.runas)
+    value = (value * 31) ^ hash(self.properties)
     return value
 
   def __repr__(self):
@@ -10387,11 +10410,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype465, _size462) = iprot.readListBegin()
-          for _i466 in xrange(_size462):
-            _elem467 = ShowCompactResponseElement()
-            _elem467.read(iprot)
-            self.compacts.append(_elem467)
+          (_etype474, _size471) = iprot.readListBegin()
+          for _i475 in xrange(_size471):
+            _elem476 = ShowCompactResponseElement()
+            _elem476.read(iprot)
+            self.compacts.append(_elem476)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10408,8 +10431,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter468 in self.compacts:
-        iter468.write(oprot)
+      for iter477 in self.compacts:
+        iter477.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10490,10 +10513,10 @@ class AddDynamicPartitions:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype472, _size469) = iprot.readListBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = iprot.readString()
-            self.partitionnames.append(_elem474)
+          (_etype481, _size478) = iprot.readListBegin()
+          for _i482 in xrange(_size478):
+            _elem483 = iprot.readString()
+            self.partitionnames.append(_elem483)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10527,8 +10550,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter475 in self.partitionnames:
-        oprot.writeString(iter475)
+      for iter484 in self.partitionnames:
+        oprot.writeString(iter484)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -10814,11 +10837,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype479, _size476) = iprot.readListBegin()
-          for _i480 in xrange(_size476):
-            _elem481 = NotificationEvent()
-            _elem481.read(iprot)
-            self.events.append(_elem481)
+          (_etype488, _size485) = iprot.readListBegin()
+          for _i489 in xrange(_size485):
+            _elem490 = NotificationEvent()
+            _elem490.read(iprot)
+            self.events.append(_elem490)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10835,8 +10858,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter482 in self.events:
-        iter482.write(oprot)
+      for iter491 in self.events:
+        iter491.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10957,10 +10980,10 @@ class InsertEventRequestData:
       if fid == 1:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype486, _size483) = iprot.readListBegin()
-          for _i487 in xrange(_size483):
-            _elem488 = iprot.readString()
-            self.filesAdded.append(_elem488)
+          (_etype495, _size492) = iprot.readListBegin()
+          for _i496 in xrange(_size492):
+            _elem497 = iprot.readString()
+            self.filesAdded.append(_elem497)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10977,8 +11000,8 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter489 in self.filesAdded:
-        oprot.writeString(iter489)
+      for iter498 in self.filesAdded:
+        oprot.writeString(iter498)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11131,10 +11154,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype493, _size490) = iprot.readListBegin()
-          for _i494 in xrange(_size490):
-            _elem495 = iprot.readString()
-            self.partitionVals.append(_elem495)
+          (_etype502, _size499) = iprot.readListBegin()
+          for _i503 in xrange(_size499):
+            _elem504 = iprot.readString()
+            self.partitionVals.append(_elem504)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11167,8 +11190,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter496 in self.partitionVals:
-        oprot.writeString(iter496)
+      for iter505 in self.partitionVals:
+        oprot.writeString(iter505)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11355,12 +11378,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype498, _vtype499, _size497 ) = iprot.readMapBegin()
-          for _i501 in xrange(_size497):
-            _key502 = iprot.readI64()
-            _val503 = MetadataPpdResult()
-            _val503.read(iprot)
-            self.metadata[_key502] = _val503
+          (_ktype507, _vtype508, _size506 ) = iprot.readMapBegin()
+          for _i510 in xrange(_size506):
+            _key511 = iprot.readI64()
+            _val512 = MetadataPpdResult()
+            _val512.read(iprot)
+            self.metadata[_key511] = _val512
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11382,9 +11405,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 kiter504,viter505 in self.metadata.items():
-        oprot.writeI64(kiter504)
-        viter505.write(oprot)
+      for kiter513,viter514 in self.metadata.items():
+        oprot.writeI64(kiter513)
+        viter514.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -11454,10 +11477,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype509, _size506) = iprot.readListBegin()
-          for _i510 in xrange(_size506):
-            _elem511 = iprot.readI64()
-            self.fileIds.append(_elem511)
+          (_etype518, _size515) = iprot.readListBegin()
+          for _i519 in xrange(_size515):
+            _elem520 = iprot.readI64()
+            self.fileIds.append(_elem520)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11489,8 +11512,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter512 in self.fileIds:
-        oprot.writeI64(iter512)
+      for iter521 in self.fileIds:
+        oprot.writeI64(iter521)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -11564,11 +11587,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype514, _vtype515, _size513 ) = iprot.readMapBegin()
-          for _i517 in xrange(_size513):
-            _key518 = iprot.readI64()
-            _val519 = iprot.readString()
-            self.metadata[_key518] = _val519
+          (_ktype523, _vtype524, _size522 ) = iprot.readMapBegin()
+          for _i526 in xrange(_size522):
+            _key527 = iprot.readI64()
+            _val528 = iprot.readString()
+            self.metadata[_key527] = _val528
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11590,9 +11613,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 kiter520,viter521 in self.metadata.items():
-        oprot.writeI64(kiter520)
-        oprot.writeString(viter521)
+      for kiter529,viter530 in self.metadata.items():
+        oprot.writeI64(kiter529)
+        oprot.writeString(viter530)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -11653,10 +11676,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype525, _size522) = iprot.readListBegin()
-          for _i526 in xrange(_size522):
-            _elem527 = iprot.readI64()
-            self.fileIds.append(_elem527)
+          (_etype534, _size531) = iprot.readListBegin()
+          for _i535 in xrange(_size531):
+            _elem536 = iprot.readI64()
+            self.fileIds.append(_elem536)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11673,8 +11696,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter528 in self.fileIds:
-        oprot.writeI64(iter528)
+      for iter537 in self.fileIds:
+        oprot.writeI64(iter537)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11780,20 +11803,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype532, _size529) = iprot.readListBegin()
-          for _i533 in xrange(_size529):
-            _elem534 = iprot.readI64()
-            self.fileIds.append(_elem534)
+          (_etype541, _size538) = iprot.readListBegin()
+          for _i542 in xrange(_size538):
+            _elem543 = iprot.readI64()
+            self.fileIds.append(_elem543)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype538, _size535) = iprot.readListBegin()
-          for _i539 in xrange(_size535):
-            _elem540 = iprot.readString()
-            self.metadata.append(_elem540)
+          (_etype547, _size544) = iprot.readListBegin()
+          for _i548 in xrange(_size544):
+            _elem549 = iprot.readString()
+            self.metadata.append(_elem549)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11815,15 +11838,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter541 in self.fileIds:
-        oprot.writeI64(iter541)
+      for iter550 in self.fileIds:
+        oprot.writeI64(iter550)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter542 in self.metadata:
-        oprot.writeString(iter542)
+      for iter551 in self.metadata:
+        oprot.writeString(iter551)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -11931,10 +11954,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype546, _size543) = iprot.readListBegin()
-          for _i547 in xrange(_size543):
-            _elem548 = iprot.readI64()
-            self.fileIds.append(_elem548)
+          (_etype555, _size552) = iprot.readListBegin()
+          for _i556 in xrange(_size552):
+            _elem557 = iprot.readI64()
+            self.fileIds.append(_elem557)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11951,8 +11974,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter549 in self.fileIds:
-        oprot.writeI64(iter549)
+      for iter558 in self.fileIds:
+        oprot.writeI64(iter558)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12181,11 +12204,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype553, _size550) = iprot.readListBegin()
-          for _i554 in xrange(_size550):
-            _elem555 = Function()
-            _elem555.read(iprot)
-            self.functions.append(_elem555)
+          (_etype562, _size559) = iprot.readListBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = Function()
+            _elem564.read(iprot)
+            self.functions.append(_elem564)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12202,8 +12225,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter556 in self.functions:
-        iter556.write(oprot)
+      for iter565 in self.functions:
+        iter565.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 82d685d..9c1b399 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -39,6 +39,7 @@ import org.apache.commons.pool.impl.GenericObjectPool;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils.StringableMap;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.StringUtils;
 
@@ -1366,6 +1367,9 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         String partName = rqst.getPartitionname();
         if (partName != null) buf.append("cq_partition, ");
         buf.append("cq_state, cq_type");
+        if (rqst.getProperties() != null) {
+          buf.append(", cq_tblproperties");
+        }
         if (rqst.getRunas() != null) buf.append(", cq_run_as");
         buf.append(") values (");
         buf.append(id);
@@ -1394,6 +1398,10 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             dbConn.rollback();
             throw new MetaException("Unexpected compaction type " + rqst.getType().toString());
         }
+        if (rqst.getProperties() != null) {
+          buf.append("', '");
+          buf.append(new StringableMap(rqst.getProperties()).toString());
+        }
         if (rqst.getRunas() != null) {
           buf.append("', '");
           buf.append(rqst.getRunas());

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/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 3fa1233..379eddc 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
@@ -3456,16 +3456,18 @@ private void constructOneLBLocationMap(FileStatus fSta,
    * @param partName name of the partition, if null table will be compacted (valid only for
    *                 non-partitioned tables).
    * @param compactType major or minor
+   * @param tblproperties the list of tblproperties to overwrite for this compaction
    * @throws HiveException
    */
-  public void compact(String dbname, String tableName, String partName,  String compactType)
+  public void compact(String dbname, String tableName, String partName, String compactType,
+                      Map<String, String> tblproperties)
       throws HiveException {
     try {
       CompactionType cr = null;
       if ("major".equals(compactType)) cr = CompactionType.MAJOR;
       else if ("minor".equals(compactType)) cr = CompactionType.MINOR;
       else throw new RuntimeException("Unknown compaction type " + compactType);
-      getMSC().compact(dbname, tableName, partName, cr);
+      getMSC().compact(dbname, tableName, partName, cr, tblproperties);
     } catch (Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw new HiveException(e);

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 767c10c..666f13b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.txn.compactor;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.JobConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.ValidTxnList;
@@ -56,6 +57,7 @@ public class Worker extends CompactorThread {
   static final private int baseThreadNum = 10002;
 
   private String name;
+  private JobConf mrJob; // the MR job for compaction
 
   /**
    * Get the hostname that this worker is run on.  Made static and public so that other classes
@@ -180,6 +182,9 @@ public class Worker extends CompactorThread {
             }
           }
           txnHandler.markCompacted(ci);
+          if (conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)) {
+            mrJob = mr.getMrJob();
+          }
         } catch (Exception e) {
           LOG.error("Caught exception while trying to compact " + ci +
               ".  Marking failed to avoid repeated failures, " + StringUtils.stringifyException(e));
@@ -213,6 +218,10 @@ public class Worker extends CompactorThread {
     setName(name.toString());
   }
 
+  public JobConf getMrJob() {
+    return mrJob;
+  }
+
   static final class StatsUpdater {
     static final private Logger LOG = LoggerFactory.getLogger(StatsUpdater.class);
 

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


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

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 13a8b71..cb5dec9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -28842,13 +28842,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list632 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list632.size);
-                  String _elem633;
-                  for (int _i634 = 0; _i634 < _list632.size; ++_i634)
+                  org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list642.size);
+                  String _elem643;
+                  for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                   {
-                    _elem633 = iprot.readString();
-                    struct.success.add(_elem633);
+                    _elem643 = iprot.readString();
+                    struct.success.add(_elem643);
                   }
                   iprot.readListEnd();
                 }
@@ -28883,9 +28883,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter635 : struct.success)
+            for (String _iter645 : struct.success)
             {
-              oprot.writeString(_iter635);
+              oprot.writeString(_iter645);
             }
             oprot.writeListEnd();
           }
@@ -28924,9 +28924,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter636 : struct.success)
+            for (String _iter646 : struct.success)
             {
-              oprot.writeString(_iter636);
+              oprot.writeString(_iter646);
             }
           }
         }
@@ -28941,13 +28941,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list637 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list637.size);
-            String _elem638;
-            for (int _i639 = 0; _i639 < _list637.size; ++_i639)
+            org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list647.size);
+            String _elem648;
+            for (int _i649 = 0; _i649 < _list647.size; ++_i649)
             {
-              _elem638 = iprot.readString();
-              struct.success.add(_elem638);
+              _elem648 = iprot.readString();
+              struct.success.add(_elem648);
             }
           }
           struct.setSuccessIsSet(true);
@@ -29601,13 +29601,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list640 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list640.size);
-                  String _elem641;
-                  for (int _i642 = 0; _i642 < _list640.size; ++_i642)
+                  org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list650.size);
+                  String _elem651;
+                  for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                   {
-                    _elem641 = iprot.readString();
-                    struct.success.add(_elem641);
+                    _elem651 = iprot.readString();
+                    struct.success.add(_elem651);
                   }
                   iprot.readListEnd();
                 }
@@ -29642,9 +29642,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter643 : struct.success)
+            for (String _iter653 : struct.success)
             {
-              oprot.writeString(_iter643);
+              oprot.writeString(_iter653);
             }
             oprot.writeListEnd();
           }
@@ -29683,9 +29683,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter644 : struct.success)
+            for (String _iter654 : struct.success)
             {
-              oprot.writeString(_iter644);
+              oprot.writeString(_iter654);
             }
           }
         }
@@ -29700,13 +29700,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list645 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list645.size);
-            String _elem646;
-            for (int _i647 = 0; _i647 < _list645.size; ++_i647)
+            org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list655.size);
+            String _elem656;
+            for (int _i657 = 0; _i657 < _list655.size; ++_i657)
             {
-              _elem646 = iprot.readString();
-              struct.success.add(_elem646);
+              _elem656 = iprot.readString();
+              struct.success.add(_elem656);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34313,16 +34313,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map648 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map648.size);
-                  String _key649;
-                  Type _val650;
-                  for (int _i651 = 0; _i651 < _map648.size; ++_i651)
+                  org.apache.thrift.protocol.TMap _map658 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map658.size);
+                  String _key659;
+                  Type _val660;
+                  for (int _i661 = 0; _i661 < _map658.size; ++_i661)
                   {
-                    _key649 = iprot.readString();
-                    _val650 = new Type();
-                    _val650.read(iprot);
-                    struct.success.put(_key649, _val650);
+                    _key659 = iprot.readString();
+                    _val660 = new Type();
+                    _val660.read(iprot);
+                    struct.success.put(_key659, _val660);
                   }
                   iprot.readMapEnd();
                 }
@@ -34357,10 +34357,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter652 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter662 : struct.success.entrySet())
             {
-              oprot.writeString(_iter652.getKey());
-              _iter652.getValue().write(oprot);
+              oprot.writeString(_iter662.getKey());
+              _iter662.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -34399,10 +34399,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter653 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter663 : struct.success.entrySet())
             {
-              oprot.writeString(_iter653.getKey());
-              _iter653.getValue().write(oprot);
+              oprot.writeString(_iter663.getKey());
+              _iter663.getValue().write(oprot);
             }
           }
         }
@@ -34417,16 +34417,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map654 = 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*_map654.size);
-            String _key655;
-            Type _val656;
-            for (int _i657 = 0; _i657 < _map654.size; ++_i657)
+            org.apache.thrift.protocol.TMap _map664 = 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*_map664.size);
+            String _key665;
+            Type _val666;
+            for (int _i667 = 0; _i667 < _map664.size; ++_i667)
             {
-              _key655 = iprot.readString();
-              _val656 = new Type();
-              _val656.read(iprot);
-              struct.success.put(_key655, _val656);
+              _key665 = iprot.readString();
+              _val666 = new Type();
+              _val666.read(iprot);
+              struct.success.put(_key665, _val666);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35461,14 +35461,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list658.size);
-                  FieldSchema _elem659;
-                  for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                  org.apache.thrift.protocol.TList _list668 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list668.size);
+                  FieldSchema _elem669;
+                  for (int _i670 = 0; _i670 < _list668.size; ++_i670)
                   {
-                    _elem659 = new FieldSchema();
-                    _elem659.read(iprot);
-                    struct.success.add(_elem659);
+                    _elem669 = new FieldSchema();
+                    _elem669.read(iprot);
+                    struct.success.add(_elem669);
                   }
                   iprot.readListEnd();
                 }
@@ -35521,9 +35521,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter661 : struct.success)
+            for (FieldSchema _iter671 : struct.success)
             {
-              _iter661.write(oprot);
+              _iter671.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35578,9 +35578,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter662 : struct.success)
+            for (FieldSchema _iter672 : struct.success)
             {
-              _iter662.write(oprot);
+              _iter672.write(oprot);
             }
           }
         }
@@ -35601,14 +35601,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list663.size);
-            FieldSchema _elem664;
-            for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+            org.apache.thrift.protocol.TList _list673 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list673.size);
+            FieldSchema _elem674;
+            for (int _i675 = 0; _i675 < _list673.size; ++_i675)
             {
-              _elem664 = new FieldSchema();
-              _elem664.read(iprot);
-              struct.success.add(_elem664);
+              _elem674 = new FieldSchema();
+              _elem674.read(iprot);
+              struct.success.add(_elem674);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36762,14 +36762,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list666.size);
-                  FieldSchema _elem667;
-                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
+                  org.apache.thrift.protocol.TList _list676 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list676.size);
+                  FieldSchema _elem677;
+                  for (int _i678 = 0; _i678 < _list676.size; ++_i678)
                   {
-                    _elem667 = new FieldSchema();
-                    _elem667.read(iprot);
-                    struct.success.add(_elem667);
+                    _elem677 = new FieldSchema();
+                    _elem677.read(iprot);
+                    struct.success.add(_elem677);
                   }
                   iprot.readListEnd();
                 }
@@ -36822,9 +36822,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter669 : struct.success)
+            for (FieldSchema _iter679 : struct.success)
             {
-              _iter669.write(oprot);
+              _iter679.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36879,9 +36879,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter670 : struct.success)
+            for (FieldSchema _iter680 : struct.success)
             {
-              _iter670.write(oprot);
+              _iter680.write(oprot);
             }
           }
         }
@@ -36902,14 +36902,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list671.size);
-            FieldSchema _elem672;
-            for (int _i673 = 0; _i673 < _list671.size; ++_i673)
+            org.apache.thrift.protocol.TList _list681 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list681.size);
+            FieldSchema _elem682;
+            for (int _i683 = 0; _i683 < _list681.size; ++_i683)
             {
-              _elem672 = new FieldSchema();
-              _elem672.read(iprot);
-              struct.success.add(_elem672);
+              _elem682 = new FieldSchema();
+              _elem682.read(iprot);
+              struct.success.add(_elem682);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37954,14 +37954,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list674.size);
-                  FieldSchema _elem675;
-                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                  org.apache.thrift.protocol.TList _list684 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list684.size);
+                  FieldSchema _elem685;
+                  for (int _i686 = 0; _i686 < _list684.size; ++_i686)
                   {
-                    _elem675 = new FieldSchema();
-                    _elem675.read(iprot);
-                    struct.success.add(_elem675);
+                    _elem685 = new FieldSchema();
+                    _elem685.read(iprot);
+                    struct.success.add(_elem685);
                   }
                   iprot.readListEnd();
                 }
@@ -38014,9 +38014,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter677 : struct.success)
+            for (FieldSchema _iter687 : struct.success)
             {
-              _iter677.write(oprot);
+              _iter687.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -38071,9 +38071,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter678 : struct.success)
+            for (FieldSchema _iter688 : struct.success)
             {
-              _iter678.write(oprot);
+              _iter688.write(oprot);
             }
           }
         }
@@ -38094,14 +38094,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list679.size);
-            FieldSchema _elem680;
-            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+            org.apache.thrift.protocol.TList _list689 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list689.size);
+            FieldSchema _elem690;
+            for (int _i691 = 0; _i691 < _list689.size; ++_i691)
             {
-              _elem680 = new FieldSchema();
-              _elem680.read(iprot);
-              struct.success.add(_elem680);
+              _elem690 = new FieldSchema();
+              _elem690.read(iprot);
+              struct.success.add(_elem690);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39255,14 +39255,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list682.size);
-                  FieldSchema _elem683;
-                  for (int _i684 = 0; _i684 < _list682.size; ++_i684)
+                  org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list692.size);
+                  FieldSchema _elem693;
+                  for (int _i694 = 0; _i694 < _list692.size; ++_i694)
                   {
-                    _elem683 = new FieldSchema();
-                    _elem683.read(iprot);
-                    struct.success.add(_elem683);
+                    _elem693 = new FieldSchema();
+                    _elem693.read(iprot);
+                    struct.success.add(_elem693);
                   }
                   iprot.readListEnd();
                 }
@@ -39315,9 +39315,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter685 : struct.success)
+            for (FieldSchema _iter695 : struct.success)
             {
-              _iter685.write(oprot);
+              _iter695.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -39372,9 +39372,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter686 : struct.success)
+            for (FieldSchema _iter696 : struct.success)
             {
-              _iter686.write(oprot);
+              _iter696.write(oprot);
             }
           }
         }
@@ -39395,14 +39395,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         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.success = new ArrayList<FieldSchema>(_list687.size);
-            FieldSchema _elem688;
-            for (int _i689 = 0; _i689 < _list687.size; ++_i689)
+            org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list697.size);
+            FieldSchema _elem698;
+            for (int _i699 = 0; _i699 < _list697.size; ++_i699)
             {
-              _elem688 = new FieldSchema();
-              _elem688.read(iprot);
-              struct.success.add(_elem688);
+              _elem698 = new FieldSchema();
+              _elem698.read(iprot);
+              struct.success.add(_elem698);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42127,14 +42127,14 @@ public class ThriftHiveMetastore {
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list690 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list690.size);
-                  SQLPrimaryKey _elem691;
-                  for (int _i692 = 0; _i692 < _list690.size; ++_i692)
+                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list700.size);
+                  SQLPrimaryKey _elem701;
+                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                   {
-                    _elem691 = new SQLPrimaryKey();
-                    _elem691.read(iprot);
-                    struct.primaryKeys.add(_elem691);
+                    _elem701 = new SQLPrimaryKey();
+                    _elem701.read(iprot);
+                    struct.primaryKeys.add(_elem701);
                   }
                   iprot.readListEnd();
                 }
@@ -42146,14 +42146,14 @@ public class ThriftHiveMetastore {
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list693 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list693.size);
-                  SQLForeignKey _elem694;
-                  for (int _i695 = 0; _i695 < _list693.size; ++_i695)
+                  org.apache.thrift.protocol.TList _list703 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list703.size);
+                  SQLForeignKey _elem704;
+                  for (int _i705 = 0; _i705 < _list703.size; ++_i705)
                   {
-                    _elem694 = new SQLForeignKey();
-                    _elem694.read(iprot);
-                    struct.foreignKeys.add(_elem694);
+                    _elem704 = new SQLForeignKey();
+                    _elem704.read(iprot);
+                    struct.foreignKeys.add(_elem704);
                   }
                   iprot.readListEnd();
                 }
@@ -42184,9 +42184,9 @@ public class ThriftHiveMetastore {
           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 _iter696 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter706 : struct.primaryKeys)
             {
-              _iter696.write(oprot);
+              _iter706.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42196,9 +42196,9 @@ public class ThriftHiveMetastore {
           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 _iter697 : struct.foreignKeys)
+            for (SQLForeignKey _iter707 : struct.foreignKeys)
             {
-              _iter697.write(oprot);
+              _iter707.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42238,18 +42238,18 @@ public class ThriftHiveMetastore {
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter698 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter708 : struct.primaryKeys)
             {
-              _iter698.write(oprot);
+              _iter708.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter699 : struct.foreignKeys)
+            for (SQLForeignKey _iter709 : struct.foreignKeys)
             {
-              _iter699.write(oprot);
+              _iter709.write(oprot);
             }
           }
         }
@@ -42266,28 +42266,28 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list700 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list700.size);
-            SQLPrimaryKey _elem701;
-            for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+            org.apache.thrift.protocol.TList _list710 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list710.size);
+            SQLPrimaryKey _elem711;
+            for (int _i712 = 0; _i712 < _list710.size; ++_i712)
             {
-              _elem701 = new SQLPrimaryKey();
-              _elem701.read(iprot);
-              struct.primaryKeys.add(_elem701);
+              _elem711 = new SQLPrimaryKey();
+              _elem711.read(iprot);
+              struct.primaryKeys.add(_elem711);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list703.size);
-            SQLForeignKey _elem704;
-            for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+            org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list713.size);
+            SQLForeignKey _elem714;
+            for (int _i715 = 0; _i715 < _list713.size; ++_i715)
             {
-              _elem704 = new SQLForeignKey();
-              _elem704.read(iprot);
-              struct.foreignKeys.add(_elem704);
+              _elem714 = new SQLForeignKey();
+              _elem714.read(iprot);
+              struct.foreignKeys.add(_elem714);
             }
           }
           struct.setForeignKeysIsSet(true);
@@ -48486,13 +48486,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list706.size);
-                  String _elem707;
-                  for (int _i708 = 0; _i708 < _list706.size; ++_i708)
+                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list716.size);
+                  String _elem717;
+                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
                   {
-                    _elem707 = iprot.readString();
-                    struct.success.add(_elem707);
+                    _elem717 = iprot.readString();
+                    struct.success.add(_elem717);
                   }
                   iprot.readListEnd();
                 }
@@ -48527,9 +48527,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter709 : struct.success)
+            for (String _iter719 : struct.success)
             {
-              oprot.writeString(_iter709);
+              oprot.writeString(_iter719);
             }
             oprot.writeListEnd();
           }
@@ -48568,9 +48568,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter710 : struct.success)
+            for (String _iter720 : struct.success)
             {
-              oprot.writeString(_iter710);
+              oprot.writeString(_iter720);
             }
           }
         }
@@ -48585,13 +48585,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list711.size);
-            String _elem712;
-            for (int _i713 = 0; _i713 < _list711.size; ++_i713)
+            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list721.size);
+            String _elem722;
+            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
             {
-              _elem712 = iprot.readString();
-              struct.success.add(_elem712);
+              _elem722 = iprot.readString();
+              struct.success.add(_elem722);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49096,13 +49096,13 @@ public class ThriftHiveMetastore {
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list714.size);
-                  String _elem715;
-                  for (int _i716 = 0; _i716 < _list714.size; ++_i716)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list724.size);
+                  String _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem715 = iprot.readString();
-                    struct.tbl_types.add(_elem715);
+                    _elem725 = iprot.readString();
+                    struct.tbl_types.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -49138,9 +49138,9 @@ public class ThriftHiveMetastore {
           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 _iter717 : struct.tbl_types)
+            for (String _iter727 : struct.tbl_types)
             {
-              oprot.writeString(_iter717);
+              oprot.writeString(_iter727);
             }
             oprot.writeListEnd();
           }
@@ -49183,9 +49183,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter718 : struct.tbl_types)
+            for (String _iter728 : struct.tbl_types)
             {
-              oprot.writeString(_iter718);
+              oprot.writeString(_iter728);
             }
           }
         }
@@ -49205,13 +49205,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list719.size);
-            String _elem720;
-            for (int _i721 = 0; _i721 < _list719.size; ++_i721)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list729.size);
+            String _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem720 = iprot.readString();
-              struct.tbl_types.add(_elem720);
+              _elem730 = iprot.readString();
+              struct.tbl_types.add(_elem730);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -49617,14 +49617,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list722.size);
-                  TableMeta _elem723;
-                  for (int _i724 = 0; _i724 < _list722.size; ++_i724)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list732.size);
+                  TableMeta _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem723 = new TableMeta();
-                    _elem723.read(iprot);
-                    struct.success.add(_elem723);
+                    _elem733 = new TableMeta();
+                    _elem733.read(iprot);
+                    struct.success.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -49659,9 +49659,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter725 : struct.success)
+            for (TableMeta _iter735 : struct.success)
             {
-              _iter725.write(oprot);
+              _iter735.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -49700,9 +49700,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter726 : struct.success)
+            for (TableMeta _iter736 : struct.success)
             {
-              _iter726.write(oprot);
+              _iter736.write(oprot);
             }
           }
         }
@@ -49717,14 +49717,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list727.size);
-            TableMeta _elem728;
-            for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list737.size);
+            TableMeta _elem738;
+            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
             {
-              _elem728 = new TableMeta();
-              _elem728.read(iprot);
-              struct.success.add(_elem728);
+              _elem738 = new TableMeta();
+              _elem738.read(iprot);
+              struct.success.add(_elem738);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50490,13 +50490,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list730 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list730.size);
-                  String _elem731;
-                  for (int _i732 = 0; _i732 < _list730.size; ++_i732)
+                  org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list740.size);
+                  String _elem741;
+                  for (int _i742 = 0; _i742 < _list740.size; ++_i742)
                   {
-                    _elem731 = iprot.readString();
-                    struct.success.add(_elem731);
+                    _elem741 = iprot.readString();
+                    struct.success.add(_elem741);
                   }
                   iprot.readListEnd();
                 }
@@ -50531,9 +50531,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter733 : struct.success)
+            for (String _iter743 : struct.success)
             {
-              oprot.writeString(_iter733);
+              oprot.writeString(_iter743);
             }
             oprot.writeListEnd();
           }
@@ -50572,9 +50572,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter734 : struct.success)
+            for (String _iter744 : struct.success)
             {
-              oprot.writeString(_iter734);
+              oprot.writeString(_iter744);
             }
           }
         }
@@ -50589,13 +50589,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list735.size);
-            String _elem736;
-            for (int _i737 = 0; _i737 < _list735.size; ++_i737)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list745.size);
+            String _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem736 = iprot.readString();
-              struct.success.add(_elem736);
+              _elem746 = iprot.readString();
+              struct.success.add(_elem746);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52048,13 +52048,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list738.size);
-                  String _elem739;
-                  for (int _i740 = 0; _i740 < _list738.size; ++_i740)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list748.size);
+                  String _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem739 = iprot.readString();
-                    struct.tbl_names.add(_elem739);
+                    _elem749 = iprot.readString();
+                    struct.tbl_names.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -52085,9 +52085,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter741 : struct.tbl_names)
+            for (String _iter751 : struct.tbl_names)
             {
-              oprot.writeString(_iter741);
+              oprot.writeString(_iter751);
             }
             oprot.writeListEnd();
           }
@@ -52124,9 +52124,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter742 : struct.tbl_names)
+            for (String _iter752 : struct.tbl_names)
             {
-              oprot.writeString(_iter742);
+              oprot.writeString(_iter752);
             }
           }
         }
@@ -52142,13 +52142,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list743.size);
-            String _elem744;
-            for (int _i745 = 0; _i745 < _list743.size; ++_i745)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list753.size);
+            String _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem744 = iprot.readString();
-              struct.tbl_names.add(_elem744);
+              _elem754 = iprot.readString();
+              struct.tbl_names.add(_elem754);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -52716,14 +52716,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list746 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list746.size);
-                  Table _elem747;
-                  for (int _i748 = 0; _i748 < _list746.size; ++_i748)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list756.size);
+                  Table _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem747 = new Table();
-                    _elem747.read(iprot);
-                    struct.success.add(_elem747);
+                    _elem757 = new Table();
+                    _elem757.read(iprot);
+                    struct.success.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -52776,9 +52776,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter749 : struct.success)
+            for (Table _iter759 : struct.success)
             {
-              _iter749.write(oprot);
+              _iter759.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52833,9 +52833,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter750 : struct.success)
+            for (Table _iter760 : struct.success)
             {
-              _iter750.write(oprot);
+              _iter760.write(oprot);
             }
           }
         }
@@ -52856,14 +52856,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list751 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list751.size);
-            Table _elem752;
-            for (int _i753 = 0; _i753 < _list751.size; ++_i753)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list761.size);
+            Table _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem752 = new Table();
-              _elem752.read(iprot);
-              struct.success.add(_elem752);
+              _elem762 = new Table();
+              _elem762.read(iprot);
+              struct.success.add(_elem762);
             }
           }
           struct.setSuccessIsSet(true);
@@ -54009,13 +54009,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list754 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list754.size);
-                  String _elem755;
-                  for (int _i756 = 0; _i756 < _list754.size; ++_i756)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list764.size);
+                  String _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem755 = iprot.readString();
-                    struct.success.add(_elem755);
+                    _elem765 = iprot.readString();
+                    struct.success.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -54068,9 +54068,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter757 : struct.success)
+            for (String _iter767 : struct.success)
             {
-              oprot.writeString(_iter757);
+              oprot.writeString(_iter767);
             }
             oprot.writeListEnd();
           }
@@ -54125,9 +54125,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter758 : struct.success)
+            for (String _iter768 : struct.success)
             {
-              oprot.writeString(_iter758);
+              oprot.writeString(_iter768);
             }
           }
         }
@@ -54148,13 +54148,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list759 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list759.size);
-            String _elem760;
-            for (int _i761 = 0; _i761 < _list759.size; ++_i761)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list769.size);
+            String _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem760 = iprot.readString();
-              struct.success.add(_elem760);
+              _elem770 = iprot.readString();
+              struct.success.add(_elem770);
             }
           }
           struct.setSuccessIsSet(true);
@@ -60013,14 +60013,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list762 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list762.size);
-                  Partition _elem763;
-                  for (int _i764 = 0; _i764 < _list762.size; ++_i764)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list772.size);
+                  Partition _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem763 = new Partition();
-                    _elem763.read(iprot);
-                    struct.new_parts.add(_elem763);
+                    _elem773 = new Partition();
+                    _elem773.read(iprot);
+                    struct.new_parts.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -60046,9 +60046,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter765 : struct.new_parts)
+            for (Partition _iter775 : struct.new_parts)
             {
-              _iter765.write(oprot);
+              _iter775.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -60079,9 +60079,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter766 : struct.new_parts)
+            for (Partition _iter776 : struct.new_parts)
             {
-              _iter766.write(oprot);
+              _iter776.write(oprot);
             }
           }
         }
@@ -60093,14 +60093,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list767 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list767.size);
-            Partition _elem768;
-            for (int _i769 = 0; _i769 < _list767.size; ++_i769)
+            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list777.size);
+            Partition _elem778;
+            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
             {
-              _elem768 = new Partition();
-              _elem768.read(iprot);
-              struct.new_parts.add(_elem768);
+              _elem778 = new Partition();
+              _elem778.read(iprot);
+              struct.new_parts.add(_elem778);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -61101,14 +61101,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list770 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list770.size);
-                  PartitionSpec _elem771;
-                  for (int _i772 = 0; _i772 < _list770.size; ++_i772)
+                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list780.size);
+                  PartitionSpec _elem781;
+                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                   {
-                    _elem771 = new PartitionSpec();
-                    _elem771.read(iprot);
-                    struct.new_parts.add(_elem771);
+                    _elem781 = new PartitionSpec();
+                    _elem781.read(iprot);
+                    struct.new_parts.add(_elem781);
                   }
                   iprot.readListEnd();
                 }
@@ -61134,9 +61134,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter773 : struct.new_parts)
+            for (PartitionSpec _iter783 : struct.new_parts)
             {
-              _iter773.write(oprot);
+              _iter783.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -61167,9 +61167,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter774 : struct.new_parts)
+            for (PartitionSpec _iter784 : struct.new_parts)
             {
-              _iter774.write(oprot);
+              _iter784.write(oprot);
             }
           }
         }
@@ -61181,14 +61181,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list775 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list775.size);
-            PartitionSpec _elem776;
-            for (int _i777 = 0; _i777 < _list775.size; ++_i777)
+            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list785.size);
+            PartitionSpec _elem786;
+            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
             {
-              _elem776 = new PartitionSpec();
-              _elem776.read(iprot);
-              struct.new_parts.add(_elem776);
+              _elem786 = new PartitionSpec();
+              _elem786.read(iprot);
+              struct.new_parts.add(_elem786);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -62364,13 +62364,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list778 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list778.size);
-                  String _elem779;
-                  for (int _i780 = 0; _i780 < _list778.size; ++_i780)
+                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list788.size);
+                  String _elem789;
+                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                   {
-                    _elem779 = iprot.readString();
-                    struct.part_vals.add(_elem779);
+                    _elem789 = iprot.readString();
+                    struct.part_vals.add(_elem789);
                   }
                   iprot.readListEnd();
                 }
@@ -62406,9 +62406,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter781 : struct.part_vals)
+            for (String _iter791 : struct.part_vals)
             {
-              oprot.writeString(_iter781);
+              oprot.writeString(_iter791);
             }
             oprot.writeListEnd();
           }
@@ -62451,9 +62451,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter782 : struct.part_vals)
+            for (String _iter792 : struct.part_vals)
             {
-              oprot.writeString(_iter782);
+              oprot.writeString(_iter792);
             }
           }
         }
@@ -62473,13 +62473,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list783 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list783.size);
-            String _elem784;
-            for (int _i785 = 0; _i785 < _list783.size; ++_i785)
+            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list793.size);
+            String _elem794;
+            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
             {
-              _elem784 = iprot.readString();
-              struct.part_vals.add(_elem784);
+              _elem794 = iprot.readString();
+              struct.part_vals.add(_elem794);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -64788,13 +64788,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list786 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list786.size);
-                  String _elem787;
-                  for (int _i788 = 0; _i788 < _list786.size; ++_i788)
+                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list796.size);
+                  String _elem797;
+                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                   {
-                    _elem787 = iprot.readString();
-                    struct.part_vals.add(_elem787);
+                    _elem797 = iprot.readString();
+                    struct.part_vals.add(_elem797);
                   }
                   iprot.readListEnd();
                 }
@@ -64839,9 +64839,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter789 : struct.part_vals)
+            for (String _iter799 : struct.part_vals)
             {
-              oprot.writeString(_iter789);
+              oprot.writeString(_iter799);
             }
             oprot.writeListEnd();
           }
@@ -64892,9 +64892,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter790 : struct.part_vals)
+            for (String _iter800 : struct.part_vals)
             {
-              oprot.writeString(_iter790);
+              oprot.writeString(_iter800);
             }
           }
         }
@@ -64917,13 +64917,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list791 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list791.size);
-            String _elem792;
-            for (int _i793 = 0; _i793 < _list791.size; ++_i793)
+            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list801.size);
+            String _elem802;
+            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
             {
-              _elem792 = iprot.readString();
-              struct.part_vals.add(_elem792);
+              _elem802 = iprot.readString();
+              struct.part_vals.add(_elem802);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -68793,13 +68793,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list794 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list794.size);
-                  String _elem795;
-                  for (int _i796 = 0; _i796 < _list794.size; ++_i796)
+                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list804.size);
+                  String _elem805;
+                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
                   {
-                    _elem795 = iprot.readString();
-                    struct.part_vals.add(_elem795);
+                    _elem805 = iprot.readString();
+                    struct.part_vals.add(_elem805);
                   }
                   iprot.readListEnd();
                 }
@@ -68843,9 +68843,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter797 : struct.part_vals)
+            for (String _iter807 : struct.part_vals)
             {
-              oprot.writeString(_iter797);
+              oprot.writeString(_iter807);
             }
             oprot.writeListEnd();
           }
@@ -68894,9 +68894,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter798 : struct.part_vals)
+            for (String _iter808 : struct.part_vals)
             {
-              oprot.writeString(_iter798);
+              oprot.writeString(_iter808);
             }
           }
         }
@@ -68919,13 +68919,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list799 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list799.size);
-            String _elem800;
-            for (int _i801 = 0; _i801 < _list799.size; ++_i801)
+            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list809.size);
+            String _elem810;
+            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
             {
-              _elem800 = iprot.readString();
-              struct.part_vals.add(_elem800);
+              _elem810 = iprot.readString();
+              struct.part_vals.add(_elem810);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -70164,13 +70164,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list802 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list802.size);
-                  String _elem803;
-                  for (int _i804 = 0; _i804 < _list802.size; ++_i804)
+                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list812.size);
+                  String _elem813;
+                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
                   {
-                    _elem803 = iprot.readString();
-                    struct.part_vals.add(_elem803);
+                    _elem813 = iprot.readString();
+                    struct.part_vals.add(_elem813);
                   }
                   iprot.readListEnd();
                 }
@@ -70223,9 +70223,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter805 : struct.part_vals)
+            for (String _iter815 : struct.part_vals)
             {
-              oprot.writeString(_iter805);
+              oprot.writeString(_iter815);
             }
             oprot.writeListEnd();
           }
@@ -70282,9 +70282,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter806 : struct.part_vals)
+            for (String _iter816 : struct.part_vals)
             {
-              oprot.writeString(_iter806);
+              oprot.writeString(_iter816);
             }
           }
         }
@@ -70310,13 +70310,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list807 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list807.size);
-            String _elem808;
-            for (int _i809 = 0; _i809 < _list807.size; ++_i809)
+            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list817.size);
+            String _elem818;
+            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
             {
-              _elem808 = iprot.readString();
-              struct.part_vals.add(_elem808);
+              _elem818 = iprot.readString();
+              struct.part_vals.add(_elem818);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -74918,13 +74918,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list810 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list810.size);
-                  String _elem811;
-                  for (int _i812 = 0; _i812 < _list810.size; ++_i812)
+                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list820.size);
+                  String _elem821;
+                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                   {
-                    _elem811 = iprot.readString();
-                    struct.part_vals.add(_elem811);
+                    _elem821 = iprot.readString();
+                    struct.part_vals.add(_elem821);
                   }
                   iprot.readListEnd();
                 }
@@ -74960,9 +74960,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter813 : struct.part_vals)
+            for (String _iter823 : struct.part_vals)
             {
-              oprot.writeString(_iter813);
+              oprot.writeString(_iter823);
             }
             oprot.writeListEnd();
           }
@@ -75005,9 +75005,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter814 : struct.part_vals)
+            for (String _iter824 : struct.part_vals)
             {
-              oprot.writeString(_iter814);
+              oprot.writeString(_iter824);
             }
           }
         }
@@ -75027,13 +75027,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list815 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list815.size);
-            String _elem816;
-            for (int _i817 = 0; _i817 < _list815.size; ++_i817)
+            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list825.size);
+            String _elem826;
+            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
             {
-              _elem816 = iprot.readString();
-              struct.part_vals.add(_elem816);
+              _elem826 = iprot.readString();
+              struct.part_vals.add(_elem826);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -76251,15 +76251,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map818 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map818.size);
-                  String _key819;
-                  String _val820;
-                  for (int _i821 = 0; _i821 < _map818.size; ++_i821)
+                  org.apache.thrift.protocol.TMap _map828 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map828.size);
+                  String _key829;
+                  String _val830;
+                  for (int _i831 = 0; _i831 < _map828.size; ++_i831)
                   {
-                    _key819 = iprot.readString();
-                    _val820 = iprot.readString();
-                    struct.partitionSpecs.put(_key819, _val820);
+                    _key829 = iprot.readString();
+                    _val830 = iprot.readString();
+                    struct.partitionSpecs.put(_key829, _val830);
                   }
                   iprot.readMapEnd();
                 }
@@ -76317,10 +76317,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter822 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter832 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter822.getKey());
-              oprot.writeString(_iter822.getValue());
+              oprot.writeString(_iter832.getKey());
+              oprot.writeString(_iter832.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -76383,10 +76383,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter823 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter833 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter823.getKey());
-              oprot.writeString(_iter823.getValue());
+              oprot.writeString(_iter833.getKey());
+              oprot.writeString(_iter833.getValue());
             }
           }
         }
@@ -76410,15 +76410,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map824 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map824.size);
-            String _key825;
-            String _val826;
-            for (int _i827 = 0; _i827 < _map824.size; ++_i827)
+            org.apache.thrift.protocol.TMap _map834 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map834.size);
+            String _key835;
+            String _val836;
+            for (int _i837 = 0; _i837 < _map834.size; ++_i837)
             {
-              _key825 = iprot.readString();
-              _val826 = iprot.readString();
-              struct.partitionSpecs.put(_key825, _val826);
+              _key835 = iprot.readString();
+              _val836 = iprot.readString();
+              struct.partitionSpecs.put(_key835, _val836);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -77864,15 +77864,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map828 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map828.size);
-                  String _key829;
-                  String _val830;
-                  for (int _i831 = 0; _i831 < _map828.size; ++_i831)
+                  org.apache.thrift.protocol.TMap _map838 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map838.size);
+                  String _key839;
+                  String _val840;
+                  for (int _i841 = 0; _i841 < _map838.size; ++_i841)
                   {
-                    _key829 = iprot.readString();
-                    _val830 = iprot.readString();
-                    struct.partitionSpecs.put(_key829, _val830);
+                    _key839 = iprot.readString();
+                    _val840 = iprot.readString();
+                    struct.partitionSpecs.put(_key839, _val840);
                   }
                   iprot.readMapEnd();
                 }
@@ -77930,10 +77930,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter832 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter842 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter832.getKey());
-              oprot.writeString(_iter832.getValue());
+              oprot.writeString(_iter842.getKey());
+              oprot.writeString(_iter842.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -77996,10 +77996,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter833 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter843 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter833.getKey());
-              oprot.writeString(_iter833.getValue());
+              oprot.writeString(_iter843.getKey());
+              oprot.writeString(_iter843.getValue());
             }
           }
         }
@@ -78023,15 +78023,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map834 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partitionSpecs = new HashMap<String,String>(2*_map834.size);
-            String _key835;
-            String _val836;
-            for (int _i837 = 0; _i837 < _map834.size; ++_i837)
+            org.apache.thrift.protocol.TMap _map844 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map844.size);
+            String _key845;
+            String _val846;
+            for (int _i847 = 0; _i847 < _map844.size; ++_i847)
             {
-              _key835 = iprot.readString();
-              _val836 = iprot.readString();
-              struct.partitionSpecs.put(_key835, _val836);
+              _key845 = iprot.readString();
+              _val846 = iprot.readString();
+              struct.partitionSpecs.put(_key845, _val846);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -78696,14 +78696,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list838 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list838.size);
-                  Partition _elem839;
-                  for (int _i840 = 0; _i840 < _list838.size; ++_i840)
+                  org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list848.size);
+                  Partition _elem849;
+                  for (int _i850 = 0; _i850 < _list848.size; ++_i850)
                   {
-                    _elem839 = new Partition();
-                    _elem839.read(iprot);
-                    struct.success.add(_elem839);
+                    _elem849 = new Partition();
+                    _elem849.read(iprot);
+                    struct.success.add(_elem849);
                   }
                   iprot.readListEnd();
                 }
@@ -78765,9 +78765,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter841 : struct.success)
+            for (Partition _iter851 : struct.success)
             {
-              _iter841.write(oprot);
+              _iter851.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -78830,9 +78830,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter842 : struct.success)
+            for (Partition _iter852 : struct.success)
             {
-              _iter842.write(oprot);
+              _iter852.write(oprot);
             }
           }
         }
@@ -78856,14 +78856,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list843 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list843.size);
-            Partition _elem844;
-            for (int _i845 = 0; _i845 < _list843.size; ++_i845)
+            org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list853.size);
+            Partition _elem854;
+            for (int _i855 = 0; _i855 < _list853.size; ++_i855)
             {
-              _elem844 = new Partition();
-              _elem844.read(iprot);
-              struct.success.add(_elem844);
+              _elem854 = new Partition();
+              _elem854.read(iprot);
+              struct.success.add(_elem854);
             }
           }
           struct.setSuccessIsSet(true);
@@ -79562,13 +79562,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list846 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list846.size);
-                  String _elem847;
-                  for (int _i848 = 0; _i848 < _list846.size; ++_i848)
+                  org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list856.size);
+                  String _elem857;
+                  for (int _i858 = 0; _i858 < _list856.size; ++_i858)
                   {
-                    _elem847 = iprot.readString();
-                    struct.part_vals.add(_elem847);
+                    _elem857 = iprot.readString();
+                    struct.part_vals.add(_elem857);
                   }
                   iprot.readListEnd();
                 }
@@ -79588,13 +79588,13 @@ public class ThriftHiveMetastore {
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list849 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list849.size);
-                  String _elem850;
-                  for (int _i851 = 0; _i851 < _list849.size; ++_i851)
+                  org.apache.thrift.protocol.TList _list859 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list859.size);
+                  String _elem860;
+                  for (int _i861 = 0; _i861 < _list859.size; ++_i861)
                   {
-                    _elem850 = iprot.readString();
-                    struct.group_names.add(_elem850);
+                    _elem860 = iprot.readString();
+                    struct.group_names.add(_elem860);
                   }
                   iprot.readListEnd();
                 }
@@ -79630,9 +79630,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter852 : struct.part_vals)
+            for (String _iter862 : struct.part_vals)
             {
-              oprot.writeString(_iter852);
+              oprot.writeString(_iter862);
             }
             oprot.writeListEnd();
           }
@@ -79647,9 +79647,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter853 : struct.group_names)
+            for (String _iter863 : struct.group_names)
             {
-              oprot.writeString(_iter853);
+              oprot.writeString(_iter863);
             }
             oprot.writeListEnd();
           }
@@ -79698,9 +79698,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter854 : struct.part_vals)
+            for (String _iter864 : struct.part_vals)
             {
-              oprot.writeString(_iter854);
+              oprot.writeString(_iter864);
             }
           }
         }
@@ -79710,9 +79710,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter855 : struct.group_names)
+            for (String _iter865 : struct.group_names)
             {
-              oprot.writeString(_iter855);
+              oprot.writeString(_iter865);
             }
           }
         }
@@ -79732,13 +79732,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list856 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list856.size);
-            String _elem857;
-            for (int _i858 = 0; _i858 < _list856.size; ++_i858)
+            org.apache.thrift.protocol.TList _list866 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list866.size);
+            String _elem867;
+            for (int _i868 = 0; _i868 < _list866.size; ++_i868)
             {
-              _elem857 = iprot.readString();
-              struct.part_vals.add(_elem857);
+              _elem867 = iprot.readString();
+              struct.part_vals.add(_elem867);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -79749,13 +79749,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol

<TRUNCATED>

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

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


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

Branch: refs/heads/master
Commit: c57a59611fa168ee38c6ee0ee60b1d6c4994f9f8
Parents: 793681c
Author: Wei Zheng <we...@apache.org>
Authored: Fri May 27 11:20:14 2016 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Fri May 27 11:20:14 2016 -0700

----------------------------------------------------------------------
 .../hive/ql/txn/compactor/TestCompactor.java    |  161 ++
 metastore/if/hive_metastore.thrift              |    1 +
 .../upgrade/derby/036-HIVE-13354.derby.sql      |    2 +
 .../derby/hive-txn-schema-1.3.0.derby.sql       |    2 +
 .../derby/hive-txn-schema-2.1.0.derby.sql       |    2 +
 .../derby/upgrade-1.2.0-to-1.3.0.derby.sql      |    1 +
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |    1 +
 .../upgrade/mssql/021-HIVE-13354.mssql.sql      |    2 +
 .../upgrade/mssql/hive-schema-1.3.0.mssql.sql   |    2 +
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |    2 +
 .../mssql/upgrade-1.2.0-to-1.3.0.mssql.sql      |    1 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |    1 +
 .../upgrade/mysql/036-HIVE-13354.mysql.sql      |    2 +
 .../mysql/hive-txn-schema-1.3.0.mysql.sql       |    2 +
 .../mysql/hive-txn-schema-2.1.0.mysql.sql       |    2 +
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |    1 +
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |    1 +
 .../upgrade/oracle/036-HIVE-13354.oracle.sql    |    2 +
 .../oracle/hive-txn-schema-1.3.0.oracle.sql     |    2 +
 .../oracle/hive-txn-schema-2.1.0.oracle.sql     |    2 +
 .../oracle/upgrade-1.2.0-to-1.3.0.oracle.sql    |    1 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |    1 +
 .../postgres/035-HIVE-13354.postgres.sql        |    2 +
 .../postgres/hive-txn-schema-1.3.0.postgres.sql |    2 +
 .../postgres/hive-txn-schema-2.1.0.postgres.sql |    2 +
 .../upgrade-1.2.0-to-1.3.0.postgres.sql         |    1 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2020 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  980 ++++----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   10 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/CompactionRequest.java   |  166 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 2188 +++++++++---------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1232 +++++-----
 .../src/gen/thrift/gen-php/metastore/Types.php  |  315 +--
 .../hive_metastore/ThriftHiveMetastore.py       |  830 +++----
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  197 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   14 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   18 +
 .../hive/metastore/txn/CompactionInfo.java      |   30 +-
 .../metastore/txn/CompactionTxnHandler.java     |   11 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |    2 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    8 +
 .../hadoop/hive/metastore/txn/TxnUtils.java     |   56 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |    2 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    6 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |    5 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    4 +-
 .../hive/ql/plan/AlterTableSimpleDesc.java      |    8 +
 .../hive/ql/txn/compactor/CompactorMR.java      |  121 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |   22 +-
 .../hadoop/hive/ql/txn/compactor/Worker.java    |    9 +
 .../hive/ql/txn/compactor/TestWorker.java       |    9 +-
 65 files changed, 4780 insertions(+), 4145 deletions(-)
----------------------------------------------------------------------


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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
index 9c730af..08be7fd 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
@@ -1,5 +1,6 @@
 -- Upgrade MetaStore schema from 2.0.0 to 2.1.0
 RUN '034-HIVE-13076.derby.sql';
 RUN '035-HIVE-13395.derby.sql';
+RUN '036-HIVE-13354.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
index 833e1a5..dc30d24 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;
 
 :r 019-HIVE-13076.mssql.sql
 :r 020-HIVE-13395.mssql.sql
+:r 021-HIVE-13354.mssql.sql
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
index 57191cc..226f4b4 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';
 
 SOURCE 034-HIVE-13076.mysql.sql;
 SOURCE 035-HIVE-13395.mysql.sql;
+SOURCE 036-HIVE-13354.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
index e27047e..13bdcdd 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;
 
 @034-HIVE-13076.oracle.sql;
 @035-HIVE-13395.oracle.sql;
+@036-HIVE-13354.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
index a7293f7..fa3e723 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
@@ -2,6 +2,7 @@ SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0';
 
 \i 033-HIVE-13076.postgres.sql;
 \i 034-HIVE-13395.postgres.sql;
+\i 035-HIVE-13354.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.1.0', "VERSION_COMMENT"='Hive release version 2.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0';


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

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index f8ca7cd..79460a8 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -14413,6 +14413,11 @@ void CompactionRequest::__set_runas(const std::string& val) {
 __isset.runas = true;
 }
 
+void CompactionRequest::__set_properties(const std::map<std::string, std::string> & val) {
+  this->properties = val;
+__isset.properties = true;
+}
+
 uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -14479,6 +14484,29 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_MAP) {
+          {
+            this->properties.clear();
+            uint32_t _size603;
+            ::apache::thrift::protocol::TType _ktype604;
+            ::apache::thrift::protocol::TType _vtype605;
+            xfer += iprot->readMapBegin(_ktype604, _vtype605, _size603);
+            uint32_t _i607;
+            for (_i607 = 0; _i607 < _size603; ++_i607)
+            {
+              std::string _key608;
+              xfer += iprot->readString(_key608);
+              std::string& _val609 = this->properties[_key608];
+              xfer += iprot->readString(_val609);
+            }
+            xfer += iprot->readMapEnd();
+          }
+          this->__isset.properties = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -14524,6 +14552,20 @@ uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeString(this->runas);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.properties) {
+    xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
+    {
+      xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
+      std::map<std::string, std::string> ::const_iterator _iter610;
+      for (_iter610 = this->properties.begin(); _iter610 != this->properties.end(); ++_iter610)
+      {
+        xfer += oprot->writeString(_iter610->first);
+        xfer += oprot->writeString(_iter610->second);
+      }
+      xfer += oprot->writeMapEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -14536,24 +14578,27 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.partitionname, b.partitionname);
   swap(a.type, b.type);
   swap(a.runas, b.runas);
+  swap(a.properties, b.properties);
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other603) {
-  dbname = other603.dbname;
-  tablename = other603.tablename;
-  partitionname = other603.partitionname;
-  type = other603.type;
-  runas = other603.runas;
-  __isset = other603.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other604) {
-  dbname = other604.dbname;
-  tablename = other604.tablename;
-  partitionname = other604.partitionname;
-  type = other604.type;
-  runas = other604.runas;
-  __isset = other604.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other611) {
+  dbname = other611.dbname;
+  tablename = other611.tablename;
+  partitionname = other611.partitionname;
+  type = other611.type;
+  runas = other611.runas;
+  properties = other611.properties;
+  __isset = other611.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other612) {
+  dbname = other612.dbname;
+  tablename = other612.tablename;
+  partitionname = other612.partitionname;
+  type = other612.type;
+  runas = other612.runas;
+  properties = other612.properties;
+  __isset = other612.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -14564,6 +14609,7 @@ void CompactionRequest::printTo(std::ostream& out) const {
   out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "<null>"));
   out << ", " << "type=" << to_string(type);
   out << ", " << "runas="; (__isset.runas ? (out << to_string(runas)) : (out << "<null>"));
+  out << ", " << "properties="; (__isset.properties ? (out << to_string(properties)) : (out << "<null>"));
   out << ")";
 }
 
@@ -14616,11 +14662,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other605) {
-  (void) other605;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other613) {
+  (void) other613;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other606) {
-  (void) other606;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other614) {
+  (void) other614;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -14741,9 +14787,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast607;
-          xfer += iprot->readI32(ecast607);
-          this->type = (CompactionType::type)ecast607;
+          int32_t ecast615;
+          xfer += iprot->readI32(ecast615);
+          this->type = (CompactionType::type)ecast615;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -14916,35 +14962,35 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other608) {
-  dbname = other608.dbname;
-  tablename = other608.tablename;
-  partitionname = other608.partitionname;
-  type = other608.type;
-  state = other608.state;
-  workerid = other608.workerid;
-  start = other608.start;
-  runAs = other608.runAs;
-  hightestTxnId = other608.hightestTxnId;
-  metaInfo = other608.metaInfo;
-  endTime = other608.endTime;
-  hadoopJobId = other608.hadoopJobId;
-  __isset = other608.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other609) {
-  dbname = other609.dbname;
-  tablename = other609.tablename;
-  partitionname = other609.partitionname;
-  type = other609.type;
-  state = other609.state;
-  workerid = other609.workerid;
-  start = other609.start;
-  runAs = other609.runAs;
-  hightestTxnId = other609.hightestTxnId;
-  metaInfo = other609.metaInfo;
-  endTime = other609.endTime;
-  hadoopJobId = other609.hadoopJobId;
-  __isset = other609.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other616) {
+  dbname = other616.dbname;
+  tablename = other616.tablename;
+  partitionname = other616.partitionname;
+  type = other616.type;
+  state = other616.state;
+  workerid = other616.workerid;
+  start = other616.start;
+  runAs = other616.runAs;
+  hightestTxnId = other616.hightestTxnId;
+  metaInfo = other616.metaInfo;
+  endTime = other616.endTime;
+  hadoopJobId = other616.hadoopJobId;
+  __isset = other616.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other617) {
+  dbname = other617.dbname;
+  tablename = other617.tablename;
+  partitionname = other617.partitionname;
+  type = other617.type;
+  state = other617.state;
+  workerid = other617.workerid;
+  start = other617.start;
+  runAs = other617.runAs;
+  hightestTxnId = other617.hightestTxnId;
+  metaInfo = other617.metaInfo;
+  endTime = other617.endTime;
+  hadoopJobId = other617.hadoopJobId;
+  __isset = other617.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -15000,14 +15046,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size610;
-            ::apache::thrift::protocol::TType _etype613;
-            xfer += iprot->readListBegin(_etype613, _size610);
-            this->compacts.resize(_size610);
-            uint32_t _i614;
-            for (_i614 = 0; _i614 < _size610; ++_i614)
+            uint32_t _size618;
+            ::apache::thrift::protocol::TType _etype621;
+            xfer += iprot->readListBegin(_etype621, _size618);
+            this->compacts.resize(_size618);
+            uint32_t _i622;
+            for (_i622 = 0; _i622 < _size618; ++_i622)
             {
-              xfer += this->compacts[_i614].read(iprot);
+              xfer += this->compacts[_i622].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15038,10 +15084,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter615;
-    for (_iter615 = this->compacts.begin(); _iter615 != this->compacts.end(); ++_iter615)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter623;
+    for (_iter623 = this->compacts.begin(); _iter623 != this->compacts.end(); ++_iter623)
     {
-      xfer += (*_iter615).write(oprot);
+      xfer += (*_iter623).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15057,11 +15103,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other616) {
-  compacts = other616.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other624) {
+  compacts = other624.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other617) {
-  compacts = other617.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other625) {
+  compacts = other625.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -15150,14 +15196,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size618;
-            ::apache::thrift::protocol::TType _etype621;
-            xfer += iprot->readListBegin(_etype621, _size618);
-            this->partitionnames.resize(_size618);
-            uint32_t _i622;
-            for (_i622 = 0; _i622 < _size618; ++_i622)
+            uint32_t _size626;
+            ::apache::thrift::protocol::TType _etype629;
+            xfer += iprot->readListBegin(_etype629, _size626);
+            this->partitionnames.resize(_size626);
+            uint32_t _i630;
+            for (_i630 = 0; _i630 < _size626; ++_i630)
             {
-              xfer += iprot->readString(this->partitionnames[_i622]);
+              xfer += iprot->readString(this->partitionnames[_i630]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15168,9 +15214,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast623;
-          xfer += iprot->readI32(ecast623);
-          this->operationType = (DataOperationType::type)ecast623;
+          int32_t ecast631;
+          xfer += iprot->readI32(ecast631);
+          this->operationType = (DataOperationType::type)ecast631;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -15216,10 +15262,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter624;
-    for (_iter624 = this->partitionnames.begin(); _iter624 != this->partitionnames.end(); ++_iter624)
+    std::vector<std::string> ::const_iterator _iter632;
+    for (_iter632 = this->partitionnames.begin(); _iter632 != this->partitionnames.end(); ++_iter632)
     {
-      xfer += oprot->writeString((*_iter624));
+      xfer += oprot->writeString((*_iter632));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15245,21 +15291,21 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other625) {
-  txnid = other625.txnid;
-  dbname = other625.dbname;
-  tablename = other625.tablename;
-  partitionnames = other625.partitionnames;
-  operationType = other625.operationType;
-  __isset = other625.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other626) {
-  txnid = other626.txnid;
-  dbname = other626.dbname;
-  tablename = other626.tablename;
-  partitionnames = other626.partitionnames;
-  operationType = other626.operationType;
-  __isset = other626.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other633) {
+  txnid = other633.txnid;
+  dbname = other633.dbname;
+  tablename = other633.tablename;
+  partitionnames = other633.partitionnames;
+  operationType = other633.operationType;
+  __isset = other633.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other634) {
+  txnid = other634.txnid;
+  dbname = other634.dbname;
+  tablename = other634.tablename;
+  partitionnames = other634.partitionnames;
+  operationType = other634.operationType;
+  __isset = other634.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -15365,15 +15411,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other627) {
-  lastEvent = other627.lastEvent;
-  maxEvents = other627.maxEvents;
-  __isset = other627.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other635) {
+  lastEvent = other635.lastEvent;
+  maxEvents = other635.maxEvents;
+  __isset = other635.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other628) {
-  lastEvent = other628.lastEvent;
-  maxEvents = other628.maxEvents;
-  __isset = other628.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other636) {
+  lastEvent = other636.lastEvent;
+  maxEvents = other636.maxEvents;
+  __isset = other636.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -15555,23 +15601,23 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other629) {
-  eventId = other629.eventId;
-  eventTime = other629.eventTime;
-  eventType = other629.eventType;
-  dbName = other629.dbName;
-  tableName = other629.tableName;
-  message = other629.message;
-  __isset = other629.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other630) {
-  eventId = other630.eventId;
-  eventTime = other630.eventTime;
-  eventType = other630.eventType;
-  dbName = other630.dbName;
-  tableName = other630.tableName;
-  message = other630.message;
-  __isset = other630.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other637) {
+  eventId = other637.eventId;
+  eventTime = other637.eventTime;
+  eventType = other637.eventType;
+  dbName = other637.dbName;
+  tableName = other637.tableName;
+  message = other637.message;
+  __isset = other637.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other638) {
+  eventId = other638.eventId;
+  eventTime = other638.eventTime;
+  eventType = other638.eventType;
+  dbName = other638.dbName;
+  tableName = other638.tableName;
+  message = other638.message;
+  __isset = other638.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -15621,14 +15667,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size631;
-            ::apache::thrift::protocol::TType _etype634;
-            xfer += iprot->readListBegin(_etype634, _size631);
-            this->events.resize(_size631);
-            uint32_t _i635;
-            for (_i635 = 0; _i635 < _size631; ++_i635)
+            uint32_t _size639;
+            ::apache::thrift::protocol::TType _etype642;
+            xfer += iprot->readListBegin(_etype642, _size639);
+            this->events.resize(_size639);
+            uint32_t _i643;
+            for (_i643 = 0; _i643 < _size639; ++_i643)
             {
-              xfer += this->events[_i635].read(iprot);
+              xfer += this->events[_i643].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15659,10 +15705,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter636;
-    for (_iter636 = this->events.begin(); _iter636 != this->events.end(); ++_iter636)
+    std::vector<NotificationEvent> ::const_iterator _iter644;
+    for (_iter644 = this->events.begin(); _iter644 != this->events.end(); ++_iter644)
     {
-      xfer += (*_iter636).write(oprot);
+      xfer += (*_iter644).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -15678,11 +15724,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other637) {
-  events = other637.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other645) {
+  events = other645.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other638) {
-  events = other638.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other646) {
+  events = other646.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -15764,11 +15810,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other639) {
-  eventId = other639.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other647) {
+  eventId = other647.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other640) {
-  eventId = other640.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other648) {
+  eventId = other648.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -15813,14 +15859,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size641;
-            ::apache::thrift::protocol::TType _etype644;
-            xfer += iprot->readListBegin(_etype644, _size641);
-            this->filesAdded.resize(_size641);
-            uint32_t _i645;
-            for (_i645 = 0; _i645 < _size641; ++_i645)
+            uint32_t _size649;
+            ::apache::thrift::protocol::TType _etype652;
+            xfer += iprot->readListBegin(_etype652, _size649);
+            this->filesAdded.resize(_size649);
+            uint32_t _i653;
+            for (_i653 = 0; _i653 < _size649; ++_i653)
             {
-              xfer += iprot->readString(this->filesAdded[_i645]);
+              xfer += iprot->readString(this->filesAdded[_i653]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15851,10 +15897,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter646;
-    for (_iter646 = this->filesAdded.begin(); _iter646 != this->filesAdded.end(); ++_iter646)
+    std::vector<std::string> ::const_iterator _iter654;
+    for (_iter654 = this->filesAdded.begin(); _iter654 != this->filesAdded.end(); ++_iter654)
     {
-      xfer += oprot->writeString((*_iter646));
+      xfer += oprot->writeString((*_iter654));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15870,11 +15916,11 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.filesAdded, b.filesAdded);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other647) {
-  filesAdded = other647.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other655) {
+  filesAdded = other655.filesAdded;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other648) {
-  filesAdded = other648.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other656) {
+  filesAdded = other656.filesAdded;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -15954,13 +16000,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other649) {
-  insertData = other649.insertData;
-  __isset = other649.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other657) {
+  insertData = other657.insertData;
+  __isset = other657.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other650) {
-  insertData = other650.insertData;
-  __isset = other650.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other658) {
+  insertData = other658.insertData;
+  __isset = other658.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -16057,14 +16103,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size651;
-            ::apache::thrift::protocol::TType _etype654;
-            xfer += iprot->readListBegin(_etype654, _size651);
-            this->partitionVals.resize(_size651);
-            uint32_t _i655;
-            for (_i655 = 0; _i655 < _size651; ++_i655)
+            uint32_t _size659;
+            ::apache::thrift::protocol::TType _etype662;
+            xfer += iprot->readListBegin(_etype662, _size659);
+            this->partitionVals.resize(_size659);
+            uint32_t _i663;
+            for (_i663 = 0; _i663 < _size659; ++_i663)
             {
-              xfer += iprot->readString(this->partitionVals[_i655]);
+              xfer += iprot->readString(this->partitionVals[_i663]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16116,10 +16162,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionVals.size()));
-      std::vector<std::string> ::const_iterator _iter656;
-      for (_iter656 = this->partitionVals.begin(); _iter656 != this->partitionVals.end(); ++_iter656)
+      std::vector<std::string> ::const_iterator _iter664;
+      for (_iter664 = this->partitionVals.begin(); _iter664 != this->partitionVals.end(); ++_iter664)
       {
-        xfer += oprot->writeString((*_iter656));
+        xfer += oprot->writeString((*_iter664));
       }
       xfer += oprot->writeListEnd();
     }
@@ -16140,21 +16186,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other657) {
-  successful = other657.successful;
-  data = other657.data;
-  dbName = other657.dbName;
-  tableName = other657.tableName;
-  partitionVals = other657.partitionVals;
-  __isset = other657.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other658) {
-  successful = other658.successful;
-  data = other658.data;
-  dbName = other658.dbName;
-  tableName = other658.tableName;
-  partitionVals = other658.partitionVals;
-  __isset = other658.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other665) {
+  successful = other665.successful;
+  data = other665.data;
+  dbName = other665.dbName;
+  tableName = other665.tableName;
+  partitionVals = other665.partitionVals;
+  __isset = other665.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other666) {
+  successful = other666.successful;
+  data = other666.data;
+  dbName = other666.dbName;
+  tableName = other666.tableName;
+  partitionVals = other666.partitionVals;
+  __isset = other666.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -16217,11 +16263,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other659) {
-  (void) other659;
+FireEventResponse::FireEventResponse(const FireEventResponse& other667) {
+  (void) other667;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other660) {
-  (void) other660;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other668) {
+  (void) other668;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -16321,15 +16367,15 @@ void swap(MetadataPpdResult &a, MetadataPpdResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other661) {
-  metadata = other661.metadata;
-  includeBitset = other661.includeBitset;
-  __isset = other661.__isset;
+MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other669) {
+  metadata = other669.metadata;
+  includeBitset = other669.includeBitset;
+  __isset = other669.__isset;
 }
-MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other662) {
-  metadata = other662.metadata;
-  includeBitset = other662.includeBitset;
-  __isset = other662.__isset;
+MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other670) {
+  metadata = other670.metadata;
+  includeBitset = other670.includeBitset;
+  __isset = other670.__isset;
   return *this;
 }
 void MetadataPpdResult::printTo(std::ostream& out) const {
@@ -16380,17 +16426,17 @@ uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size663;
-            ::apache::thrift::protocol::TType _ktype664;
-            ::apache::thrift::protocol::TType _vtype665;
-            xfer += iprot->readMapBegin(_ktype664, _vtype665, _size663);
-            uint32_t _i667;
-            for (_i667 = 0; _i667 < _size663; ++_i667)
+            uint32_t _size671;
+            ::apache::thrift::protocol::TType _ktype672;
+            ::apache::thrift::protocol::TType _vtype673;
+            xfer += iprot->readMapBegin(_ktype672, _vtype673, _size671);
+            uint32_t _i675;
+            for (_i675 = 0; _i675 < _size671; ++_i675)
             {
-              int64_t _key668;
-              xfer += iprot->readI64(_key668);
-              MetadataPpdResult& _val669 = this->metadata[_key668];
-              xfer += _val669.read(iprot);
+              int64_t _key676;
+              xfer += iprot->readI64(_key676);
+              MetadataPpdResult& _val677 = this->metadata[_key676];
+              xfer += _val677.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -16431,11 +16477,11 @@ uint32_t GetFileMetadataByExprResult::write(::apache::thrift::protocol::TProtoco
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter670;
-    for (_iter670 = this->metadata.begin(); _iter670 != this->metadata.end(); ++_iter670)
+    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter678;
+    for (_iter678 = this->metadata.begin(); _iter678 != this->metadata.end(); ++_iter678)
     {
-      xfer += oprot->writeI64(_iter670->first);
-      xfer += _iter670->second.write(oprot);
+      xfer += oprot->writeI64(_iter678->first);
+      xfer += _iter678->second.write(oprot);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -16456,13 +16502,13 @@ void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other671) {
-  metadata = other671.metadata;
-  isSupported = other671.isSupported;
+GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other679) {
+  metadata = other679.metadata;
+  isSupported = other679.isSupported;
 }
-GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other672) {
-  metadata = other672.metadata;
-  isSupported = other672.isSupported;
+GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other680) {
+  metadata = other680.metadata;
+  isSupported = other680.isSupported;
   return *this;
 }
 void GetFileMetadataByExprResult::printTo(std::ostream& out) const {
@@ -16523,14 +16569,14 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size673;
-            ::apache::thrift::protocol::TType _etype676;
-            xfer += iprot->readListBegin(_etype676, _size673);
-            this->fileIds.resize(_size673);
-            uint32_t _i677;
-            for (_i677 = 0; _i677 < _size673; ++_i677)
+            uint32_t _size681;
+            ::apache::thrift::protocol::TType _etype684;
+            xfer += iprot->readListBegin(_etype684, _size681);
+            this->fileIds.resize(_size681);
+            uint32_t _i685;
+            for (_i685 = 0; _i685 < _size681; ++_i685)
             {
-              xfer += iprot->readI64(this->fileIds[_i677]);
+              xfer += iprot->readI64(this->fileIds[_i685]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16557,9 +16603,9 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast678;
-          xfer += iprot->readI32(ecast678);
-          this->type = (FileMetadataExprType::type)ecast678;
+          int32_t ecast686;
+          xfer += iprot->readI32(ecast686);
+          this->type = (FileMetadataExprType::type)ecast686;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -16589,10 +16635,10 @@ uint32_t GetFileMetadataByExprRequest::write(::apache::thrift::protocol::TProtoc
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter679;
-    for (_iter679 = this->fileIds.begin(); _iter679 != this->fileIds.end(); ++_iter679)
+    std::vector<int64_t> ::const_iterator _iter687;
+    for (_iter687 = this->fileIds.begin(); _iter687 != this->fileIds.end(); ++_iter687)
     {
-      xfer += oprot->writeI64((*_iter679));
+      xfer += oprot->writeI64((*_iter687));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16626,19 +16672,19 @@ void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other680) {
-  fileIds = other680.fileIds;
-  expr = other680.expr;
-  doGetFooters = other680.doGetFooters;
-  type = other680.type;
-  __isset = other680.__isset;
+GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other688) {
+  fileIds = other688.fileIds;
+  expr = other688.expr;
+  doGetFooters = other688.doGetFooters;
+  type = other688.type;
+  __isset = other688.__isset;
 }
-GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other681) {
-  fileIds = other681.fileIds;
-  expr = other681.expr;
-  doGetFooters = other681.doGetFooters;
-  type = other681.type;
-  __isset = other681.__isset;
+GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other689) {
+  fileIds = other689.fileIds;
+  expr = other689.expr;
+  doGetFooters = other689.doGetFooters;
+  type = other689.type;
+  __isset = other689.__isset;
   return *this;
 }
 void GetFileMetadataByExprRequest::printTo(std::ostream& out) const {
@@ -16691,17 +16737,17 @@ uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size682;
-            ::apache::thrift::protocol::TType _ktype683;
-            ::apache::thrift::protocol::TType _vtype684;
-            xfer += iprot->readMapBegin(_ktype683, _vtype684, _size682);
-            uint32_t _i686;
-            for (_i686 = 0; _i686 < _size682; ++_i686)
+            uint32_t _size690;
+            ::apache::thrift::protocol::TType _ktype691;
+            ::apache::thrift::protocol::TType _vtype692;
+            xfer += iprot->readMapBegin(_ktype691, _vtype692, _size690);
+            uint32_t _i694;
+            for (_i694 = 0; _i694 < _size690; ++_i694)
             {
-              int64_t _key687;
-              xfer += iprot->readI64(_key687);
-              std::string& _val688 = this->metadata[_key687];
-              xfer += iprot->readBinary(_val688);
+              int64_t _key695;
+              xfer += iprot->readI64(_key695);
+              std::string& _val696 = this->metadata[_key695];
+              xfer += iprot->readBinary(_val696);
             }
             xfer += iprot->readMapEnd();
           }
@@ -16742,11 +16788,11 @@ uint32_t GetFileMetadataResult::write(::apache::thrift::protocol::TProtocol* opr
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_I64, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::map<int64_t, std::string> ::const_iterator _iter689;
-    for (_iter689 = this->metadata.begin(); _iter689 != this->metadata.end(); ++_iter689)
+    std::map<int64_t, std::string> ::const_iterator _iter697;
+    for (_iter697 = this->metadata.begin(); _iter697 != this->metadata.end(); ++_iter697)
     {
-      xfer += oprot->writeI64(_iter689->first);
-      xfer += oprot->writeBinary(_iter689->second);
+      xfer += oprot->writeI64(_iter697->first);
+      xfer += oprot->writeBinary(_iter697->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -16767,13 +16813,13 @@ void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other690) {
-  metadata = other690.metadata;
-  isSupported = other690.isSupported;
+GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other698) {
+  metadata = other698.metadata;
+  isSupported = other698.isSupported;
 }
-GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other691) {
-  metadata = other691.metadata;
-  isSupported = other691.isSupported;
+GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other699) {
+  metadata = other699.metadata;
+  isSupported = other699.isSupported;
   return *this;
 }
 void GetFileMetadataResult::printTo(std::ostream& out) const {
@@ -16819,14 +16865,14 @@ uint32_t GetFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size692;
-            ::apache::thrift::protocol::TType _etype695;
-            xfer += iprot->readListBegin(_etype695, _size692);
-            this->fileIds.resize(_size692);
-            uint32_t _i696;
-            for (_i696 = 0; _i696 < _size692; ++_i696)
+            uint32_t _size700;
+            ::apache::thrift::protocol::TType _etype703;
+            xfer += iprot->readListBegin(_etype703, _size700);
+            this->fileIds.resize(_size700);
+            uint32_t _i704;
+            for (_i704 = 0; _i704 < _size700; ++_i704)
             {
-              xfer += iprot->readI64(this->fileIds[_i696]);
+              xfer += iprot->readI64(this->fileIds[_i704]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16857,10 +16903,10 @@ uint32_t GetFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter697;
-    for (_iter697 = this->fileIds.begin(); _iter697 != this->fileIds.end(); ++_iter697)
+    std::vector<int64_t> ::const_iterator _iter705;
+    for (_iter705 = this->fileIds.begin(); _iter705 != this->fileIds.end(); ++_iter705)
     {
-      xfer += oprot->writeI64((*_iter697));
+      xfer += oprot->writeI64((*_iter705));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16876,11 +16922,11 @@ void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other698) {
-  fileIds = other698.fileIds;
+GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other706) {
+  fileIds = other706.fileIds;
 }
-GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other699) {
-  fileIds = other699.fileIds;
+GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other707) {
+  fileIds = other707.fileIds;
   return *this;
 }
 void GetFileMetadataRequest::printTo(std::ostream& out) const {
@@ -16939,11 +16985,11 @@ void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) {
   (void) b;
 }
 
-PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other700) {
-  (void) other700;
+PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other708) {
+  (void) other708;
 }
-PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other701) {
-  (void) other701;
+PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other709) {
+  (void) other709;
   return *this;
 }
 void PutFileMetadataResult::printTo(std::ostream& out) const {
@@ -16997,14 +17043,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size702;
-            ::apache::thrift::protocol::TType _etype705;
-            xfer += iprot->readListBegin(_etype705, _size702);
-            this->fileIds.resize(_size702);
-            uint32_t _i706;
-            for (_i706 = 0; _i706 < _size702; ++_i706)
+            uint32_t _size710;
+            ::apache::thrift::protocol::TType _etype713;
+            xfer += iprot->readListBegin(_etype713, _size710);
+            this->fileIds.resize(_size710);
+            uint32_t _i714;
+            for (_i714 = 0; _i714 < _size710; ++_i714)
             {
-              xfer += iprot->readI64(this->fileIds[_i706]);
+              xfer += iprot->readI64(this->fileIds[_i714]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17017,14 +17063,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->metadata.clear();
-            uint32_t _size707;
-            ::apache::thrift::protocol::TType _etype710;
-            xfer += iprot->readListBegin(_etype710, _size707);
-            this->metadata.resize(_size707);
-            uint32_t _i711;
-            for (_i711 = 0; _i711 < _size707; ++_i711)
+            uint32_t _size715;
+            ::apache::thrift::protocol::TType _etype718;
+            xfer += iprot->readListBegin(_etype718, _size715);
+            this->metadata.resize(_size715);
+            uint32_t _i719;
+            for (_i719 = 0; _i719 < _size715; ++_i719)
             {
-              xfer += iprot->readBinary(this->metadata[_i711]);
+              xfer += iprot->readBinary(this->metadata[_i719]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17035,9 +17081,9 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast712;
-          xfer += iprot->readI32(ecast712);
-          this->type = (FileMetadataExprType::type)ecast712;
+          int32_t ecast720;
+          xfer += iprot->readI32(ecast720);
+          this->type = (FileMetadataExprType::type)ecast720;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17067,10 +17113,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter713;
-    for (_iter713 = this->fileIds.begin(); _iter713 != this->fileIds.end(); ++_iter713)
+    std::vector<int64_t> ::const_iterator _iter721;
+    for (_iter721 = this->fileIds.begin(); _iter721 != this->fileIds.end(); ++_iter721)
     {
-      xfer += oprot->writeI64((*_iter713));
+      xfer += oprot->writeI64((*_iter721));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17079,10 +17125,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::vector<std::string> ::const_iterator _iter714;
-    for (_iter714 = this->metadata.begin(); _iter714 != this->metadata.end(); ++_iter714)
+    std::vector<std::string> ::const_iterator _iter722;
+    for (_iter722 = this->metadata.begin(); _iter722 != this->metadata.end(); ++_iter722)
     {
-      xfer += oprot->writeBinary((*_iter714));
+      xfer += oprot->writeBinary((*_iter722));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17106,17 +17152,17 @@ void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other715) {
-  fileIds = other715.fileIds;
-  metadata = other715.metadata;
-  type = other715.type;
-  __isset = other715.__isset;
+PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other723) {
+  fileIds = other723.fileIds;
+  metadata = other723.metadata;
+  type = other723.type;
+  __isset = other723.__isset;
 }
-PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other716) {
-  fileIds = other716.fileIds;
-  metadata = other716.metadata;
-  type = other716.type;
-  __isset = other716.__isset;
+PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other724) {
+  fileIds = other724.fileIds;
+  metadata = other724.metadata;
+  type = other724.type;
+  __isset = other724.__isset;
   return *this;
 }
 void PutFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17177,11 +17223,11 @@ void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) {
   (void) b;
 }
 
-ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other717) {
-  (void) other717;
+ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other725) {
+  (void) other725;
 }
-ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other718) {
-  (void) other718;
+ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other726) {
+  (void) other726;
   return *this;
 }
 void ClearFileMetadataResult::printTo(std::ostream& out) const {
@@ -17225,14 +17271,14 @@ uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size719;
-            ::apache::thrift::protocol::TType _etype722;
-            xfer += iprot->readListBegin(_etype722, _size719);
-            this->fileIds.resize(_size719);
-            uint32_t _i723;
-            for (_i723 = 0; _i723 < _size719; ++_i723)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            this->fileIds.resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += iprot->readI64(this->fileIds[_i723]);
+              xfer += iprot->readI64(this->fileIds[_i731]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17263,10 +17309,10 @@ uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter724;
-    for (_iter724 = this->fileIds.begin(); _iter724 != this->fileIds.end(); ++_iter724)
+    std::vector<int64_t> ::const_iterator _iter732;
+    for (_iter732 = this->fileIds.begin(); _iter732 != this->fileIds.end(); ++_iter732)
     {
-      xfer += oprot->writeI64((*_iter724));
+      xfer += oprot->writeI64((*_iter732));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17282,11 +17328,11 @@ void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other725) {
-  fileIds = other725.fileIds;
+ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other733) {
+  fileIds = other733.fileIds;
 }
-ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other726) {
-  fileIds = other726.fileIds;
+ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other734) {
+  fileIds = other734.fileIds;
   return *this;
 }
 void ClearFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17368,11 +17414,11 @@ void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other727) {
-  isSupported = other727.isSupported;
+CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other735) {
+  isSupported = other735.isSupported;
 }
-CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other728) {
-  isSupported = other728.isSupported;
+CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other736) {
+  isSupported = other736.isSupported;
   return *this;
 }
 void CacheFileMetadataResult::printTo(std::ostream& out) const {
@@ -17513,19 +17559,19 @@ void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other729) {
-  dbName = other729.dbName;
-  tblName = other729.tblName;
-  partName = other729.partName;
-  isAllParts = other729.isAllParts;
-  __isset = other729.__isset;
+CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other737) {
+  dbName = other737.dbName;
+  tblName = other737.tblName;
+  partName = other737.partName;
+  isAllParts = other737.isAllParts;
+  __isset = other737.__isset;
 }
-CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other730) {
-  dbName = other730.dbName;
-  tblName = other730.tblName;
-  partName = other730.partName;
-  isAllParts = other730.isAllParts;
-  __isset = other730.__isset;
+CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other738) {
+  dbName = other738.dbName;
+  tblName = other738.tblName;
+  partName = other738.partName;
+  isAllParts = other738.isAllParts;
+  __isset = other738.__isset;
   return *this;
 }
 void CacheFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17573,14 +17619,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size731;
-            ::apache::thrift::protocol::TType _etype734;
-            xfer += iprot->readListBegin(_etype734, _size731);
-            this->functions.resize(_size731);
-            uint32_t _i735;
-            for (_i735 = 0; _i735 < _size731; ++_i735)
+            uint32_t _size739;
+            ::apache::thrift::protocol::TType _etype742;
+            xfer += iprot->readListBegin(_etype742, _size739);
+            this->functions.resize(_size739);
+            uint32_t _i743;
+            for (_i743 = 0; _i743 < _size739; ++_i743)
             {
-              xfer += this->functions[_i735].read(iprot);
+              xfer += this->functions[_i743].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17610,10 +17656,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter736;
-      for (_iter736 = this->functions.begin(); _iter736 != this->functions.end(); ++_iter736)
+      std::vector<Function> ::const_iterator _iter744;
+      for (_iter744 = this->functions.begin(); _iter744 != this->functions.end(); ++_iter744)
       {
-        xfer += (*_iter736).write(oprot);
+        xfer += (*_iter744).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -17630,13 +17676,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other737) {
-  functions = other737.functions;
-  __isset = other737.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other745) {
+  functions = other745.functions;
+  __isset = other745.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other738) {
-  functions = other738.functions;
-  __isset = other738.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other746) {
+  functions = other746.functions;
+  __isset = other746.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -17778,19 +17824,19 @@ void swap(TableMeta &a, TableMeta &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableMeta::TableMeta(const TableMeta& other739) {
-  dbName = other739.dbName;
-  tableName = other739.tableName;
-  tableType = other739.tableType;
-  comments = other739.comments;
-  __isset = other739.__isset;
+TableMeta::TableMeta(const TableMeta& other747) {
+  dbName = other747.dbName;
+  tableName = other747.tableName;
+  tableType = other747.tableType;
+  comments = other747.comments;
+  __isset = other747.__isset;
 }
-TableMeta& TableMeta::operator=(const TableMeta& other740) {
-  dbName = other740.dbName;
-  tableName = other740.tableName;
-  tableType = other740.tableType;
-  comments = other740.comments;
-  __isset = other740.__isset;
+TableMeta& TableMeta::operator=(const TableMeta& other748) {
+  dbName = other748.dbName;
+  tableName = other748.tableName;
+  tableType = other748.tableType;
+  comments = other748.comments;
+  __isset = other748.__isset;
   return *this;
 }
 void TableMeta::printTo(std::ostream& out) const {
@@ -17873,13 +17919,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other741) : TException() {
-  message = other741.message;
-  __isset = other741.__isset;
+MetaException::MetaException(const MetaException& other749) : TException() {
+  message = other749.message;
+  __isset = other749.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other742) {
-  message = other742.message;
-  __isset = other742.__isset;
+MetaException& MetaException::operator=(const MetaException& other750) {
+  message = other750.message;
+  __isset = other750.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -17970,13 +18016,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other743) : TException() {
-  message = other743.message;
-  __isset = other743.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other751) : TException() {
+  message = other751.message;
+  __isset = other751.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other744) {
-  message = other744.message;
-  __isset = other744.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other752) {
+  message = other752.message;
+  __isset = other752.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -18067,13 +18113,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other745) : TException() {
-  message = other745.message;
-  __isset = other745.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other753) : TException() {
+  message = other753.message;
+  __isset = other753.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other746) {
-  message = other746.message;
-  __isset = other746.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other754) {
+  message = other754.message;
+  __isset = other754.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -18164,13 +18210,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other747) : TException() {
-  message = other747.message;
-  __isset = other747.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other755) : TException() {
+  message = other755.message;
+  __isset = other755.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other748) {
-  message = other748.message;
-  __isset = other748.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other756) {
+  message = other756.message;
+  __isset = other756.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -18261,13 +18307,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other749) : TException() {
-  message = other749.message;
-  __isset = other749.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other757) : TException() {
+  message = other757.message;
+  __isset = other757.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other750) {
-  message = other750.message;
-  __isset = other750.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other758) {
+  message = other758.message;
+  __isset = other758.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -18358,13 +18404,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other751) : TException() {
-  message = other751.message;
-  __isset = other751.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other759) : TException() {
+  message = other759.message;
+  __isset = other759.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other752) {
-  message = other752.message;
-  __isset = other752.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other760) {
+  message = other760.message;
+  __isset = other760.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -18455,13 +18501,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other753) : TException() {
-  message = other753.message;
-  __isset = other753.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other761) : TException() {
+  message = other761.message;
+  __isset = other761.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other754) {
-  message = other754.message;
-  __isset = other754.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other762) {
+  message = other762.message;
+  __isset = other762.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -18552,13 +18598,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other755) : TException() {
-  message = other755.message;
-  __isset = other755.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other763) : TException() {
+  message = other763.message;
+  __isset = other763.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other756) {
-  message = other756.message;
-  __isset = other756.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other764) {
+  message = other764.message;
+  __isset = other764.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -18649,13 +18695,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other757) : TException() {
-  message = other757.message;
-  __isset = other757.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other765) : TException() {
+  message = other765.message;
+  __isset = other765.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other758) {
-  message = other758.message;
-  __isset = other758.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other766) {
+  message = other766.message;
+  __isset = other766.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -18746,13 +18792,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other759) : TException() {
-  message = other759.message;
-  __isset = other759.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other767) : TException() {
+  message = other767.message;
+  __isset = other767.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other760) {
-  message = other760.message;
-  __isset = other760.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other768) {
+  message = other768.message;
+  __isset = other768.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -18843,13 +18889,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other761) : TException() {
-  message = other761.message;
-  __isset = other761.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other769) : TException() {
+  message = other769.message;
+  __isset = other769.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other762) {
-  message = other762.message;
-  __isset = other762.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other770) {
+  message = other770.message;
+  __isset = other770.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -18940,13 +18986,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other763) : TException() {
-  message = other763.message;
-  __isset = other763.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other771) : TException() {
+  message = other771.message;
+  __isset = other771.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other764) {
-  message = other764.message;
-  __isset = other764.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other772) {
+  message = other772.message;
+  __isset = other772.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -19037,13 +19083,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other765) : TException() {
-  message = other765.message;
-  __isset = other765.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other773) : TException() {
+  message = other773.message;
+  __isset = other773.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other766) {
-  message = other766.message;
-  __isset = other766.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other774) {
+  message = other774.message;
+  __isset = other774.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -19134,13 +19180,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other767) : TException() {
-  message = other767.message;
-  __isset = other767.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other775) : TException() {
+  message = other775.message;
+  __isset = other775.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other768) {
-  message = other768.message;
-  __isset = other768.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other776) {
+  message = other776.message;
+  __isset = other776.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -19231,13 +19277,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other769) : TException() {
-  message = other769.message;
-  __isset = other769.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other777) : TException() {
+  message = other777.message;
+  __isset = other777.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other770) {
-  message = other770.message;
-  __isset = other770.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other778) {
+  message = other778.message;
+  __isset = other778.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -19328,13 +19374,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other771) : TException() {
-  message = other771.message;
-  __isset = other771.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other779) : TException() {
+  message = other779.message;
+  __isset = other779.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other772) {
-  message = other772.message;
-  __isset = other772.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other780) {
+  message = other780.message;
+  __isset = other780.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 00f017b..ec81798 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -5867,9 +5867,10 @@ inline std::ostream& operator<<(std::ostream& out, const HeartbeatTxnRangeRespon
 }
 
 typedef struct _CompactionRequest__isset {
-  _CompactionRequest__isset() : partitionname(false), runas(false) {}
+  _CompactionRequest__isset() : partitionname(false), runas(false), properties(false) {}
   bool partitionname :1;
   bool runas :1;
+  bool properties :1;
 } _CompactionRequest__isset;
 
 class CompactionRequest {
@@ -5886,6 +5887,7 @@ class CompactionRequest {
   std::string partitionname;
   CompactionType::type type;
   std::string runas;
+  std::map<std::string, std::string>  properties;
 
   _CompactionRequest__isset __isset;
 
@@ -5899,6 +5901,8 @@ class CompactionRequest {
 
   void __set_runas(const std::string& val);
 
+  void __set_properties(const std::map<std::string, std::string> & val);
+
   bool operator == (const CompactionRequest & rhs) const
   {
     if (!(dbname == rhs.dbname))
@@ -5915,6 +5919,10 @@ class CompactionRequest {
       return false;
     else if (__isset.runas && !(runas == rhs.runas))
       return false;
+    if (__isset.properties != rhs.__isset.properties)
+      return false;
+    else if (__isset.properties && !(properties == rhs.properties))
+      return false;
     return true;
   }
   bool operator != (const CompactionRequest &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 734a6df..ba06a56 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -727,13 +727,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list532.size);
-                String _elem533;
-                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
+                org.apache.thrift.protocol.TList _list542 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list542.size);
+                String _elem543;
+                for (int _i544 = 0; _i544 < _list542.size; ++_i544)
                 {
-                  _elem533 = iprot.readString();
-                  struct.partitionnames.add(_elem533);
+                  _elem543 = iprot.readString();
+                  struct.partitionnames.add(_elem543);
                 }
                 iprot.readListEnd();
               }
@@ -780,9 +780,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter535 : struct.partitionnames)
+          for (String _iter545 : struct.partitionnames)
           {
-            oprot.writeString(_iter535);
+            oprot.writeString(_iter545);
           }
           oprot.writeListEnd();
         }
@@ -817,9 +817,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter536 : struct.partitionnames)
+        for (String _iter546 : struct.partitionnames)
         {
-          oprot.writeString(_iter536);
+          oprot.writeString(_iter546);
         }
       }
       BitSet optionals = new BitSet();
@@ -842,13 +842,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list537.size);
-        String _elem538;
-        for (int _i539 = 0; _i539 < _list537.size; ++_i539)
+        org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list547.size);
+        String _elem548;
+        for (int _i549 = 0; _i549 < _list547.size; ++_i549)
         {
-          _elem538 = iprot.readString();
-          struct.partitionnames.add(_elem538);
+          _elem548 = iprot.readString();
+          struct.partitionnames.add(_elem548);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index f71a3ca..ca274e6 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list616 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list616.size);
-                long _elem617;
-                for (int _i618 = 0; _i618 < _list616.size; ++_i618)
+                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list626.size);
+                long _elem627;
+                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                 {
-                  _elem617 = iprot.readI64();
-                  struct.fileIds.add(_elem617);
+                  _elem627 = iprot.readI64();
+                  struct.fileIds.add(_elem627);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
         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 _iter619 : struct.fileIds)
+          for (long _iter629 : struct.fileIds)
           {
-            oprot.writeI64(_iter619);
+            oprot.writeI64(_iter629);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter620 : struct.fileIds)
+        for (long _iter630 : struct.fileIds)
         {
-          oprot.writeI64(_iter620);
+          oprot.writeI64(_iter630);
         }
       }
     }
@@ -421,13 +421,13 @@ public class ClearFileMetadataRequest implements org.apache.thrift.TBase<ClearFi
     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 _list621 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list621.size);
-        long _elem622;
-        for (int _i623 = 0; _i623 < _list621.size; ++_i623)
+        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list631.size);
+        long _elem632;
+        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
         {
-          _elem622 = iprot.readI64();
-          struct.fileIds.add(_elem622);
+          _elem632 = iprot.readI64();
+          struct.fileIds.add(_elem632);
         }
       }
       struct.setFileIdsIsSet(true);


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

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 298384c..5a35a50 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size773;
-            ::apache::thrift::protocol::TType _etype776;
-            xfer += iprot->readListBegin(_etype776, _size773);
-            this->success.resize(_size773);
-            uint32_t _i777;
-            for (_i777 = 0; _i777 < _size773; ++_i777)
+            uint32_t _size781;
+            ::apache::thrift::protocol::TType _etype784;
+            xfer += iprot->readListBegin(_etype784, _size781);
+            this->success.resize(_size781);
+            uint32_t _i785;
+            for (_i785 = 0; _i785 < _size781; ++_i785)
             {
-              xfer += iprot->readString(this->success[_i777]);
+              xfer += iprot->readString(this->success[_i785]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter778;
-      for (_iter778 = this->success.begin(); _iter778 != this->success.end(); ++_iter778)
+      std::vector<std::string> ::const_iterator _iter786;
+      for (_iter786 = this->success.begin(); _iter786 != this->success.end(); ++_iter786)
       {
-        xfer += oprot->writeString((*_iter778));
+        xfer += oprot->writeString((*_iter786));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size779;
-            ::apache::thrift::protocol::TType _etype782;
-            xfer += iprot->readListBegin(_etype782, _size779);
-            (*(this->success)).resize(_size779);
-            uint32_t _i783;
-            for (_i783 = 0; _i783 < _size779; ++_i783)
+            uint32_t _size787;
+            ::apache::thrift::protocol::TType _etype790;
+            xfer += iprot->readListBegin(_etype790, _size787);
+            (*(this->success)).resize(_size787);
+            uint32_t _i791;
+            for (_i791 = 0; _i791 < _size787; ++_i791)
             {
-              xfer += iprot->readString((*(this->success))[_i783]);
+              xfer += iprot->readString((*(this->success))[_i791]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size784;
-            ::apache::thrift::protocol::TType _etype787;
-            xfer += iprot->readListBegin(_etype787, _size784);
-            this->success.resize(_size784);
-            uint32_t _i788;
-            for (_i788 = 0; _i788 < _size784; ++_i788)
+            uint32_t _size792;
+            ::apache::thrift::protocol::TType _etype795;
+            xfer += iprot->readListBegin(_etype795, _size792);
+            this->success.resize(_size792);
+            uint32_t _i796;
+            for (_i796 = 0; _i796 < _size792; ++_i796)
             {
-              xfer += iprot->readString(this->success[_i788]);
+              xfer += iprot->readString(this->success[_i796]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter789;
-      for (_iter789 = this->success.begin(); _iter789 != this->success.end(); ++_iter789)
+      std::vector<std::string> ::const_iterator _iter797;
+      for (_iter797 = this->success.begin(); _iter797 != this->success.end(); ++_iter797)
       {
-        xfer += oprot->writeString((*_iter789));
+        xfer += oprot->writeString((*_iter797));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size790;
-            ::apache::thrift::protocol::TType _etype793;
-            xfer += iprot->readListBegin(_etype793, _size790);
-            (*(this->success)).resize(_size790);
-            uint32_t _i794;
-            for (_i794 = 0; _i794 < _size790; ++_i794)
+            uint32_t _size798;
+            ::apache::thrift::protocol::TType _etype801;
+            xfer += iprot->readListBegin(_etype801, _size798);
+            (*(this->success)).resize(_size798);
+            uint32_t _i802;
+            for (_i802 = 0; _i802 < _size798; ++_i802)
             {
-              xfer += iprot->readString((*(this->success))[_i794]);
+              xfer += iprot->readString((*(this->success))[_i802]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size795;
-            ::apache::thrift::protocol::TType _ktype796;
-            ::apache::thrift::protocol::TType _vtype797;
-            xfer += iprot->readMapBegin(_ktype796, _vtype797, _size795);
-            uint32_t _i799;
-            for (_i799 = 0; _i799 < _size795; ++_i799)
+            uint32_t _size803;
+            ::apache::thrift::protocol::TType _ktype804;
+            ::apache::thrift::protocol::TType _vtype805;
+            xfer += iprot->readMapBegin(_ktype804, _vtype805, _size803);
+            uint32_t _i807;
+            for (_i807 = 0; _i807 < _size803; ++_i807)
             {
-              std::string _key800;
-              xfer += iprot->readString(_key800);
-              Type& _val801 = this->success[_key800];
-              xfer += _val801.read(iprot);
+              std::string _key808;
+              xfer += iprot->readString(_key808);
+              Type& _val809 = this->success[_key808];
+              xfer += _val809.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter802;
-      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
+      std::map<std::string, Type> ::const_iterator _iter810;
+      for (_iter810 = this->success.begin(); _iter810 != this->success.end(); ++_iter810)
       {
-        xfer += oprot->writeString(_iter802->first);
-        xfer += _iter802->second.write(oprot);
+        xfer += oprot->writeString(_iter810->first);
+        xfer += _iter810->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _ktype804;
-            ::apache::thrift::protocol::TType _vtype805;
-            xfer += iprot->readMapBegin(_ktype804, _vtype805, _size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size811;
+            ::apache::thrift::protocol::TType _ktype812;
+            ::apache::thrift::protocol::TType _vtype813;
+            xfer += iprot->readMapBegin(_ktype812, _vtype813, _size811);
+            uint32_t _i815;
+            for (_i815 = 0; _i815 < _size811; ++_i815)
             {
-              std::string _key808;
-              xfer += iprot->readString(_key808);
-              Type& _val809 = (*(this->success))[_key808];
-              xfer += _val809.read(iprot);
+              std::string _key816;
+              xfer += iprot->readString(_key816);
+              Type& _val817 = (*(this->success))[_key816];
+              xfer += _val817.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size810;
-            ::apache::thrift::protocol::TType _etype813;
-            xfer += iprot->readListBegin(_etype813, _size810);
-            this->success.resize(_size810);
-            uint32_t _i814;
-            for (_i814 = 0; _i814 < _size810; ++_i814)
+            uint32_t _size818;
+            ::apache::thrift::protocol::TType _etype821;
+            xfer += iprot->readListBegin(_etype821, _size818);
+            this->success.resize(_size818);
+            uint32_t _i822;
+            for (_i822 = 0; _i822 < _size818; ++_i822)
             {
-              xfer += this->success[_i814].read(iprot);
+              xfer += this->success[_i822].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter815;
-      for (_iter815 = this->success.begin(); _iter815 != this->success.end(); ++_iter815)
+      std::vector<FieldSchema> ::const_iterator _iter823;
+      for (_iter823 = this->success.begin(); _iter823 != this->success.end(); ++_iter823)
       {
-        xfer += (*_iter815).write(oprot);
+        xfer += (*_iter823).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size816;
-            ::apache::thrift::protocol::TType _etype819;
-            xfer += iprot->readListBegin(_etype819, _size816);
-            (*(this->success)).resize(_size816);
-            uint32_t _i820;
-            for (_i820 = 0; _i820 < _size816; ++_i820)
+            uint32_t _size824;
+            ::apache::thrift::protocol::TType _etype827;
+            xfer += iprot->readListBegin(_etype827, _size824);
+            (*(this->success)).resize(_size824);
+            uint32_t _i828;
+            for (_i828 = 0; _i828 < _size824; ++_i828)
             {
-              xfer += (*(this->success))[_i820].read(iprot);
+              xfer += (*(this->success))[_i828].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size821;
-            ::apache::thrift::protocol::TType _etype824;
-            xfer += iprot->readListBegin(_etype824, _size821);
-            this->success.resize(_size821);
-            uint32_t _i825;
-            for (_i825 = 0; _i825 < _size821; ++_i825)
+            uint32_t _size829;
+            ::apache::thrift::protocol::TType _etype832;
+            xfer += iprot->readListBegin(_etype832, _size829);
+            this->success.resize(_size829);
+            uint32_t _i833;
+            for (_i833 = 0; _i833 < _size829; ++_i833)
             {
-              xfer += this->success[_i825].read(iprot);
+              xfer += this->success[_i833].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter826;
-      for (_iter826 = this->success.begin(); _iter826 != this->success.end(); ++_iter826)
+      std::vector<FieldSchema> ::const_iterator _iter834;
+      for (_iter834 = this->success.begin(); _iter834 != this->success.end(); ++_iter834)
       {
-        xfer += (*_iter826).write(oprot);
+        xfer += (*_iter834).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size827;
-            ::apache::thrift::protocol::TType _etype830;
-            xfer += iprot->readListBegin(_etype830, _size827);
-            (*(this->success)).resize(_size827);
-            uint32_t _i831;
-            for (_i831 = 0; _i831 < _size827; ++_i831)
+            uint32_t _size835;
+            ::apache::thrift::protocol::TType _etype838;
+            xfer += iprot->readListBegin(_etype838, _size835);
+            (*(this->success)).resize(_size835);
+            uint32_t _i839;
+            for (_i839 = 0; _i839 < _size835; ++_i839)
             {
-              xfer += (*(this->success))[_i831].read(iprot);
+              xfer += (*(this->success))[_i839].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size832;
-            ::apache::thrift::protocol::TType _etype835;
-            xfer += iprot->readListBegin(_etype835, _size832);
-            this->success.resize(_size832);
-            uint32_t _i836;
-            for (_i836 = 0; _i836 < _size832; ++_i836)
+            uint32_t _size840;
+            ::apache::thrift::protocol::TType _etype843;
+            xfer += iprot->readListBegin(_etype843, _size840);
+            this->success.resize(_size840);
+            uint32_t _i844;
+            for (_i844 = 0; _i844 < _size840; ++_i844)
             {
-              xfer += this->success[_i836].read(iprot);
+              xfer += this->success[_i844].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter837;
-      for (_iter837 = this->success.begin(); _iter837 != this->success.end(); ++_iter837)
+      std::vector<FieldSchema> ::const_iterator _iter845;
+      for (_iter845 = this->success.begin(); _iter845 != this->success.end(); ++_iter845)
       {
-        xfer += (*_iter837).write(oprot);
+        xfer += (*_iter845).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size838;
-            ::apache::thrift::protocol::TType _etype841;
-            xfer += iprot->readListBegin(_etype841, _size838);
-            (*(this->success)).resize(_size838);
-            uint32_t _i842;
-            for (_i842 = 0; _i842 < _size838; ++_i842)
+            uint32_t _size846;
+            ::apache::thrift::protocol::TType _etype849;
+            xfer += iprot->readListBegin(_etype849, _size846);
+            (*(this->success)).resize(_size846);
+            uint32_t _i850;
+            for (_i850 = 0; _i850 < _size846; ++_i850)
             {
-              xfer += (*(this->success))[_i842].read(iprot);
+              xfer += (*(this->success))[_i850].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size843;
-            ::apache::thrift::protocol::TType _etype846;
-            xfer += iprot->readListBegin(_etype846, _size843);
-            this->success.resize(_size843);
-            uint32_t _i847;
-            for (_i847 = 0; _i847 < _size843; ++_i847)
+            uint32_t _size851;
+            ::apache::thrift::protocol::TType _etype854;
+            xfer += iprot->readListBegin(_etype854, _size851);
+            this->success.resize(_size851);
+            uint32_t _i855;
+            for (_i855 = 0; _i855 < _size851; ++_i855)
             {
-              xfer += this->success[_i847].read(iprot);
+              xfer += this->success[_i855].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter848;
-      for (_iter848 = this->success.begin(); _iter848 != this->success.end(); ++_iter848)
+      std::vector<FieldSchema> ::const_iterator _iter856;
+      for (_iter856 = this->success.begin(); _iter856 != this->success.end(); ++_iter856)
       {
-        xfer += (*_iter848).write(oprot);
+        xfer += (*_iter856).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size849;
-            ::apache::thrift::protocol::TType _etype852;
-            xfer += iprot->readListBegin(_etype852, _size849);
-            (*(this->success)).resize(_size849);
-            uint32_t _i853;
-            for (_i853 = 0; _i853 < _size849; ++_i853)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            (*(this->success)).resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += (*(this->success))[_i853].read(iprot);
+              xfer += (*(this->success))[_i861].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            this->primaryKeys.resize(_size854);
-            uint32_t _i858;
-            for (_i858 = 0; _i858 < _size854; ++_i858)
+            uint32_t _size862;
+            ::apache::thrift::protocol::TType _etype865;
+            xfer += iprot->readListBegin(_etype865, _size862);
+            this->primaryKeys.resize(_size862);
+            uint32_t _i866;
+            for (_i866 = 0; _i866 < _size862; ++_i866)
             {
-              xfer += this->primaryKeys[_i858].read(iprot);
+              xfer += this->primaryKeys[_i866].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size859;
-            ::apache::thrift::protocol::TType _etype862;
-            xfer += iprot->readListBegin(_etype862, _size859);
-            this->foreignKeys.resize(_size859);
-            uint32_t _i863;
-            for (_i863 = 0; _i863 < _size859; ++_i863)
+            uint32_t _size867;
+            ::apache::thrift::protocol::TType _etype870;
+            xfer += iprot->readListBegin(_etype870, _size867);
+            this->foreignKeys.resize(_size867);
+            uint32_t _i871;
+            for (_i871 = 0; _i871 < _size867; ++_i871)
             {
-              xfer += this->foreignKeys[_i863].read(iprot);
+              xfer += this->foreignKeys[_i871].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,10 +4578,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter864;
-    for (_iter864 = this->primaryKeys.begin(); _iter864 != this->primaryKeys.end(); ++_iter864)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter872;
+    for (_iter872 = this->primaryKeys.begin(); _iter872 != this->primaryKeys.end(); ++_iter872)
     {
-      xfer += (*_iter864).write(oprot);
+      xfer += (*_iter872).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4590,10 +4590,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter865;
-    for (_iter865 = this->foreignKeys.begin(); _iter865 != this->foreignKeys.end(); ++_iter865)
+    std::vector<SQLForeignKey> ::const_iterator _iter873;
+    for (_iter873 = this->foreignKeys.begin(); _iter873 != this->foreignKeys.end(); ++_iter873)
     {
-      xfer += (*_iter865).write(oprot);
+      xfer += (*_iter873).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4621,10 +4621,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter866;
-    for (_iter866 = (*(this->primaryKeys)).begin(); _iter866 != (*(this->primaryKeys)).end(); ++_iter866)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter874;
+    for (_iter874 = (*(this->primaryKeys)).begin(); _iter874 != (*(this->primaryKeys)).end(); ++_iter874)
     {
-      xfer += (*_iter866).write(oprot);
+      xfer += (*_iter874).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4633,10 +4633,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter867;
-    for (_iter867 = (*(this->foreignKeys)).begin(); _iter867 != (*(this->foreignKeys)).end(); ++_iter867)
+    std::vector<SQLForeignKey> ::const_iterator _iter875;
+    for (_iter875 = (*(this->foreignKeys)).begin(); _iter875 != (*(this->foreignKeys)).end(); ++_iter875)
     {
-      xfer += (*_iter867).write(oprot);
+      xfer += (*_iter875).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6055,14 +6055,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size868;
-            ::apache::thrift::protocol::TType _etype871;
-            xfer += iprot->readListBegin(_etype871, _size868);
-            this->success.resize(_size868);
-            uint32_t _i872;
-            for (_i872 = 0; _i872 < _size868; ++_i872)
+            uint32_t _size876;
+            ::apache::thrift::protocol::TType _etype879;
+            xfer += iprot->readListBegin(_etype879, _size876);
+            this->success.resize(_size876);
+            uint32_t _i880;
+            for (_i880 = 0; _i880 < _size876; ++_i880)
             {
-              xfer += iprot->readString(this->success[_i872]);
+              xfer += iprot->readString(this->success[_i880]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6101,10 +6101,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter873;
-      for (_iter873 = this->success.begin(); _iter873 != this->success.end(); ++_iter873)
+      std::vector<std::string> ::const_iterator _iter881;
+      for (_iter881 = this->success.begin(); _iter881 != this->success.end(); ++_iter881)
       {
-        xfer += oprot->writeString((*_iter873));
+        xfer += oprot->writeString((*_iter881));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6149,14 +6149,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size874;
-            ::apache::thrift::protocol::TType _etype877;
-            xfer += iprot->readListBegin(_etype877, _size874);
-            (*(this->success)).resize(_size874);
-            uint32_t _i878;
-            for (_i878 = 0; _i878 < _size874; ++_i878)
+            uint32_t _size882;
+            ::apache::thrift::protocol::TType _etype885;
+            xfer += iprot->readListBegin(_etype885, _size882);
+            (*(this->success)).resize(_size882);
+            uint32_t _i886;
+            for (_i886 = 0; _i886 < _size882; ++_i886)
             {
-              xfer += iprot->readString((*(this->success))[_i878]);
+              xfer += iprot->readString((*(this->success))[_i886]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6231,14 +6231,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size879;
-            ::apache::thrift::protocol::TType _etype882;
-            xfer += iprot->readListBegin(_etype882, _size879);
-            this->tbl_types.resize(_size879);
-            uint32_t _i883;
-            for (_i883 = 0; _i883 < _size879; ++_i883)
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            this->tbl_types.resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
             {
-              xfer += iprot->readString(this->tbl_types[_i883]);
+              xfer += iprot->readString(this->tbl_types[_i891]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6275,10 +6275,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter884;
-    for (_iter884 = this->tbl_types.begin(); _iter884 != this->tbl_types.end(); ++_iter884)
+    std::vector<std::string> ::const_iterator _iter892;
+    for (_iter892 = this->tbl_types.begin(); _iter892 != this->tbl_types.end(); ++_iter892)
     {
-      xfer += oprot->writeString((*_iter884));
+      xfer += oprot->writeString((*_iter892));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6310,10 +6310,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter885;
-    for (_iter885 = (*(this->tbl_types)).begin(); _iter885 != (*(this->tbl_types)).end(); ++_iter885)
+    std::vector<std::string> ::const_iterator _iter893;
+    for (_iter893 = (*(this->tbl_types)).begin(); _iter893 != (*(this->tbl_types)).end(); ++_iter893)
     {
-      xfer += oprot->writeString((*_iter885));
+      xfer += oprot->writeString((*_iter893));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6354,14 +6354,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size886;
-            ::apache::thrift::protocol::TType _etype889;
-            xfer += iprot->readListBegin(_etype889, _size886);
-            this->success.resize(_size886);
-            uint32_t _i890;
-            for (_i890 = 0; _i890 < _size886; ++_i890)
+            uint32_t _size894;
+            ::apache::thrift::protocol::TType _etype897;
+            xfer += iprot->readListBegin(_etype897, _size894);
+            this->success.resize(_size894);
+            uint32_t _i898;
+            for (_i898 = 0; _i898 < _size894; ++_i898)
             {
-              xfer += this->success[_i890].read(iprot);
+              xfer += this->success[_i898].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6400,10 +6400,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter891;
-      for (_iter891 = this->success.begin(); _iter891 != this->success.end(); ++_iter891)
+      std::vector<TableMeta> ::const_iterator _iter899;
+      for (_iter899 = this->success.begin(); _iter899 != this->success.end(); ++_iter899)
       {
-        xfer += (*_iter891).write(oprot);
+        xfer += (*_iter899).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6448,14 +6448,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size892;
-            ::apache::thrift::protocol::TType _etype895;
-            xfer += iprot->readListBegin(_etype895, _size892);
-            (*(this->success)).resize(_size892);
-            uint32_t _i896;
-            for (_i896 = 0; _i896 < _size892; ++_i896)
+            uint32_t _size900;
+            ::apache::thrift::protocol::TType _etype903;
+            xfer += iprot->readListBegin(_etype903, _size900);
+            (*(this->success)).resize(_size900);
+            uint32_t _i904;
+            for (_i904 = 0; _i904 < _size900; ++_i904)
             {
-              xfer += (*(this->success))[_i896].read(iprot);
+              xfer += (*(this->success))[_i904].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6593,14 +6593,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size897;
-            ::apache::thrift::protocol::TType _etype900;
-            xfer += iprot->readListBegin(_etype900, _size897);
-            this->success.resize(_size897);
-            uint32_t _i901;
-            for (_i901 = 0; _i901 < _size897; ++_i901)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            this->success.resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += iprot->readString(this->success[_i901]);
+              xfer += iprot->readString(this->success[_i909]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6639,10 +6639,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter902;
-      for (_iter902 = this->success.begin(); _iter902 != this->success.end(); ++_iter902)
+      std::vector<std::string> ::const_iterator _iter910;
+      for (_iter910 = this->success.begin(); _iter910 != this->success.end(); ++_iter910)
       {
-        xfer += oprot->writeString((*_iter902));
+        xfer += oprot->writeString((*_iter910));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6687,14 +6687,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size903;
-            ::apache::thrift::protocol::TType _etype906;
-            xfer += iprot->readListBegin(_etype906, _size903);
-            (*(this->success)).resize(_size903);
-            uint32_t _i907;
-            for (_i907 = 0; _i907 < _size903; ++_i907)
+            uint32_t _size911;
+            ::apache::thrift::protocol::TType _etype914;
+            xfer += iprot->readListBegin(_etype914, _size911);
+            (*(this->success)).resize(_size911);
+            uint32_t _i915;
+            for (_i915 = 0; _i915 < _size911; ++_i915)
             {
-              xfer += iprot->readString((*(this->success))[_i907]);
+              xfer += iprot->readString((*(this->success))[_i915]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7004,14 +7004,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size908;
-            ::apache::thrift::protocol::TType _etype911;
-            xfer += iprot->readListBegin(_etype911, _size908);
-            this->tbl_names.resize(_size908);
-            uint32_t _i912;
-            for (_i912 = 0; _i912 < _size908; ++_i912)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            this->tbl_names.resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += iprot->readString(this->tbl_names[_i912]);
+              xfer += iprot->readString(this->tbl_names[_i920]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7044,10 +7044,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter913;
-    for (_iter913 = this->tbl_names.begin(); _iter913 != this->tbl_names.end(); ++_iter913)
+    std::vector<std::string> ::const_iterator _iter921;
+    for (_iter921 = this->tbl_names.begin(); _iter921 != this->tbl_names.end(); ++_iter921)
     {
-      xfer += oprot->writeString((*_iter913));
+      xfer += oprot->writeString((*_iter921));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7075,10 +7075,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter914;
-    for (_iter914 = (*(this->tbl_names)).begin(); _iter914 != (*(this->tbl_names)).end(); ++_iter914)
+    std::vector<std::string> ::const_iterator _iter922;
+    for (_iter922 = (*(this->tbl_names)).begin(); _iter922 != (*(this->tbl_names)).end(); ++_iter922)
     {
-      xfer += oprot->writeString((*_iter914));
+      xfer += oprot->writeString((*_iter922));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7119,14 +7119,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size915;
-            ::apache::thrift::protocol::TType _etype918;
-            xfer += iprot->readListBegin(_etype918, _size915);
-            this->success.resize(_size915);
-            uint32_t _i919;
-            for (_i919 = 0; _i919 < _size915; ++_i919)
+            uint32_t _size923;
+            ::apache::thrift::protocol::TType _etype926;
+            xfer += iprot->readListBegin(_etype926, _size923);
+            this->success.resize(_size923);
+            uint32_t _i927;
+            for (_i927 = 0; _i927 < _size923; ++_i927)
             {
-              xfer += this->success[_i919].read(iprot);
+              xfer += this->success[_i927].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7181,10 +7181,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter920;
-      for (_iter920 = this->success.begin(); _iter920 != this->success.end(); ++_iter920)
+      std::vector<Table> ::const_iterator _iter928;
+      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
       {
-        xfer += (*_iter920).write(oprot);
+        xfer += (*_iter928).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7237,14 +7237,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size921;
-            ::apache::thrift::protocol::TType _etype924;
-            xfer += iprot->readListBegin(_etype924, _size921);
-            (*(this->success)).resize(_size921);
-            uint32_t _i925;
-            for (_i925 = 0; _i925 < _size921; ++_i925)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _etype932;
+            xfer += iprot->readListBegin(_etype932, _size929);
+            (*(this->success)).resize(_size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              xfer += (*(this->success))[_i925].read(iprot);
+              xfer += (*(this->success))[_i933].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7430,14 +7430,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size926;
-            ::apache::thrift::protocol::TType _etype929;
-            xfer += iprot->readListBegin(_etype929, _size926);
-            this->success.resize(_size926);
-            uint32_t _i930;
-            for (_i930 = 0; _i930 < _size926; ++_i930)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->success.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += iprot->readString(this->success[_i930]);
+              xfer += iprot->readString(this->success[_i938]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7492,10 +7492,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter931;
-      for (_iter931 = this->success.begin(); _iter931 != this->success.end(); ++_iter931)
+      std::vector<std::string> ::const_iterator _iter939;
+      for (_iter939 = this->success.begin(); _iter939 != this->success.end(); ++_iter939)
       {
-        xfer += oprot->writeString((*_iter931));
+        xfer += oprot->writeString((*_iter939));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7548,14 +7548,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size932;
-            ::apache::thrift::protocol::TType _etype935;
-            xfer += iprot->readListBegin(_etype935, _size932);
-            (*(this->success)).resize(_size932);
-            uint32_t _i936;
-            for (_i936 = 0; _i936 < _size932; ++_i936)
+            uint32_t _size940;
+            ::apache::thrift::protocol::TType _etype943;
+            xfer += iprot->readListBegin(_etype943, _size940);
+            (*(this->success)).resize(_size940);
+            uint32_t _i944;
+            for (_i944 = 0; _i944 < _size940; ++_i944)
             {
-              xfer += iprot->readString((*(this->success))[_i936]);
+              xfer += iprot->readString((*(this->success))[_i944]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8889,14 +8889,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size937;
-            ::apache::thrift::protocol::TType _etype940;
-            xfer += iprot->readListBegin(_etype940, _size937);
-            this->new_parts.resize(_size937);
-            uint32_t _i941;
-            for (_i941 = 0; _i941 < _size937; ++_i941)
+            uint32_t _size945;
+            ::apache::thrift::protocol::TType _etype948;
+            xfer += iprot->readListBegin(_etype948, _size945);
+            this->new_parts.resize(_size945);
+            uint32_t _i949;
+            for (_i949 = 0; _i949 < _size945; ++_i949)
             {
-              xfer += this->new_parts[_i941].read(iprot);
+              xfer += this->new_parts[_i949].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8925,10 +8925,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter942;
-    for (_iter942 = this->new_parts.begin(); _iter942 != this->new_parts.end(); ++_iter942)
+    std::vector<Partition> ::const_iterator _iter950;
+    for (_iter950 = this->new_parts.begin(); _iter950 != this->new_parts.end(); ++_iter950)
     {
-      xfer += (*_iter942).write(oprot);
+      xfer += (*_iter950).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8952,10 +8952,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter943;
-    for (_iter943 = (*(this->new_parts)).begin(); _iter943 != (*(this->new_parts)).end(); ++_iter943)
+    std::vector<Partition> ::const_iterator _iter951;
+    for (_iter951 = (*(this->new_parts)).begin(); _iter951 != (*(this->new_parts)).end(); ++_iter951)
     {
-      xfer += (*_iter943).write(oprot);
+      xfer += (*_iter951).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9164,14 +9164,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size944;
-            ::apache::thrift::protocol::TType _etype947;
-            xfer += iprot->readListBegin(_etype947, _size944);
-            this->new_parts.resize(_size944);
-            uint32_t _i948;
-            for (_i948 = 0; _i948 < _size944; ++_i948)
+            uint32_t _size952;
+            ::apache::thrift::protocol::TType _etype955;
+            xfer += iprot->readListBegin(_etype955, _size952);
+            this->new_parts.resize(_size952);
+            uint32_t _i956;
+            for (_i956 = 0; _i956 < _size952; ++_i956)
             {
-              xfer += this->new_parts[_i948].read(iprot);
+              xfer += this->new_parts[_i956].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9200,10 +9200,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter949;
-    for (_iter949 = this->new_parts.begin(); _iter949 != this->new_parts.end(); ++_iter949)
+    std::vector<PartitionSpec> ::const_iterator _iter957;
+    for (_iter957 = this->new_parts.begin(); _iter957 != this->new_parts.end(); ++_iter957)
     {
-      xfer += (*_iter949).write(oprot);
+      xfer += (*_iter957).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9227,10 +9227,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter950;
-    for (_iter950 = (*(this->new_parts)).begin(); _iter950 != (*(this->new_parts)).end(); ++_iter950)
+    std::vector<PartitionSpec> ::const_iterator _iter958;
+    for (_iter958 = (*(this->new_parts)).begin(); _iter958 != (*(this->new_parts)).end(); ++_iter958)
     {
-      xfer += (*_iter950).write(oprot);
+      xfer += (*_iter958).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9455,14 +9455,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size951;
-            ::apache::thrift::protocol::TType _etype954;
-            xfer += iprot->readListBegin(_etype954, _size951);
-            this->part_vals.resize(_size951);
-            uint32_t _i955;
-            for (_i955 = 0; _i955 < _size951; ++_i955)
+            uint32_t _size959;
+            ::apache::thrift::protocol::TType _etype962;
+            xfer += iprot->readListBegin(_etype962, _size959);
+            this->part_vals.resize(_size959);
+            uint32_t _i963;
+            for (_i963 = 0; _i963 < _size959; ++_i963)
             {
-              xfer += iprot->readString(this->part_vals[_i955]);
+              xfer += iprot->readString(this->part_vals[_i963]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9499,10 +9499,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter956;
-    for (_iter956 = this->part_vals.begin(); _iter956 != this->part_vals.end(); ++_iter956)
+    std::vector<std::string> ::const_iterator _iter964;
+    for (_iter964 = this->part_vals.begin(); _iter964 != this->part_vals.end(); ++_iter964)
     {
-      xfer += oprot->writeString((*_iter956));
+      xfer += oprot->writeString((*_iter964));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9534,10 +9534,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter957;
-    for (_iter957 = (*(this->part_vals)).begin(); _iter957 != (*(this->part_vals)).end(); ++_iter957)
+    std::vector<std::string> ::const_iterator _iter965;
+    for (_iter965 = (*(this->part_vals)).begin(); _iter965 != (*(this->part_vals)).end(); ++_iter965)
     {
-      xfer += oprot->writeString((*_iter957));
+      xfer += oprot->writeString((*_iter965));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10009,14 +10009,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size958;
-            ::apache::thrift::protocol::TType _etype961;
-            xfer += iprot->readListBegin(_etype961, _size958);
-            this->part_vals.resize(_size958);
-            uint32_t _i962;
-            for (_i962 = 0; _i962 < _size958; ++_i962)
+            uint32_t _size966;
+            ::apache::thrift::protocol::TType _etype969;
+            xfer += iprot->readListBegin(_etype969, _size966);
+            this->part_vals.resize(_size966);
+            uint32_t _i970;
+            for (_i970 = 0; _i970 < _size966; ++_i970)
             {
-              xfer += iprot->readString(this->part_vals[_i962]);
+              xfer += iprot->readString(this->part_vals[_i970]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10061,10 +10061,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter963;
-    for (_iter963 = this->part_vals.begin(); _iter963 != this->part_vals.end(); ++_iter963)
+    std::vector<std::string> ::const_iterator _iter971;
+    for (_iter971 = this->part_vals.begin(); _iter971 != this->part_vals.end(); ++_iter971)
     {
-      xfer += oprot->writeString((*_iter963));
+      xfer += oprot->writeString((*_iter971));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10100,10 +10100,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter964;
-    for (_iter964 = (*(this->part_vals)).begin(); _iter964 != (*(this->part_vals)).end(); ++_iter964)
+    std::vector<std::string> ::const_iterator _iter972;
+    for (_iter972 = (*(this->part_vals)).begin(); _iter972 != (*(this->part_vals)).end(); ++_iter972)
     {
-      xfer += oprot->writeString((*_iter964));
+      xfer += oprot->writeString((*_iter972));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10906,14 +10906,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size965;
-            ::apache::thrift::protocol::TType _etype968;
-            xfer += iprot->readListBegin(_etype968, _size965);
-            this->part_vals.resize(_size965);
-            uint32_t _i969;
-            for (_i969 = 0; _i969 < _size965; ++_i969)
+            uint32_t _size973;
+            ::apache::thrift::protocol::TType _etype976;
+            xfer += iprot->readListBegin(_etype976, _size973);
+            this->part_vals.resize(_size973);
+            uint32_t _i977;
+            for (_i977 = 0; _i977 < _size973; ++_i977)
             {
-              xfer += iprot->readString(this->part_vals[_i969]);
+              xfer += iprot->readString(this->part_vals[_i977]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10958,10 +10958,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter970;
-    for (_iter970 = this->part_vals.begin(); _iter970 != this->part_vals.end(); ++_iter970)
+    std::vector<std::string> ::const_iterator _iter978;
+    for (_iter978 = this->part_vals.begin(); _iter978 != this->part_vals.end(); ++_iter978)
     {
-      xfer += oprot->writeString((*_iter970));
+      xfer += oprot->writeString((*_iter978));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10997,10 +10997,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter971;
-    for (_iter971 = (*(this->part_vals)).begin(); _iter971 != (*(this->part_vals)).end(); ++_iter971)
+    std::vector<std::string> ::const_iterator _iter979;
+    for (_iter979 = (*(this->part_vals)).begin(); _iter979 != (*(this->part_vals)).end(); ++_iter979)
     {
-      xfer += oprot->writeString((*_iter971));
+      xfer += oprot->writeString((*_iter979));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11209,14 +11209,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size972;
-            ::apache::thrift::protocol::TType _etype975;
-            xfer += iprot->readListBegin(_etype975, _size972);
-            this->part_vals.resize(_size972);
-            uint32_t _i976;
-            for (_i976 = 0; _i976 < _size972; ++_i976)
+            uint32_t _size980;
+            ::apache::thrift::protocol::TType _etype983;
+            xfer += iprot->readListBegin(_etype983, _size980);
+            this->part_vals.resize(_size980);
+            uint32_t _i984;
+            for (_i984 = 0; _i984 < _size980; ++_i984)
             {
-              xfer += iprot->readString(this->part_vals[_i976]);
+              xfer += iprot->readString(this->part_vals[_i984]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11269,10 +11269,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter977;
-    for (_iter977 = this->part_vals.begin(); _iter977 != this->part_vals.end(); ++_iter977)
+    std::vector<std::string> ::const_iterator _iter985;
+    for (_iter985 = this->part_vals.begin(); _iter985 != this->part_vals.end(); ++_iter985)
     {
-      xfer += oprot->writeString((*_iter977));
+      xfer += oprot->writeString((*_iter985));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11312,10 +11312,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter978;
-    for (_iter978 = (*(this->part_vals)).begin(); _iter978 != (*(this->part_vals)).end(); ++_iter978)
+    std::vector<std::string> ::const_iterator _iter986;
+    for (_iter986 = (*(this->part_vals)).begin(); _iter986 != (*(this->part_vals)).end(); ++_iter986)
     {
-      xfer += oprot->writeString((*_iter978));
+      xfer += oprot->writeString((*_iter986));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12321,14 +12321,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size979;
-            ::apache::thrift::protocol::TType _etype982;
-            xfer += iprot->readListBegin(_etype982, _size979);
-            this->part_vals.resize(_size979);
-            uint32_t _i983;
-            for (_i983 = 0; _i983 < _size979; ++_i983)
+            uint32_t _size987;
+            ::apache::thrift::protocol::TType _etype990;
+            xfer += iprot->readListBegin(_etype990, _size987);
+            this->part_vals.resize(_size987);
+            uint32_t _i991;
+            for (_i991 = 0; _i991 < _size987; ++_i991)
             {
-              xfer += iprot->readString(this->part_vals[_i983]);
+              xfer += iprot->readString(this->part_vals[_i991]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12365,10 +12365,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter984;
-    for (_iter984 = this->part_vals.begin(); _iter984 != this->part_vals.end(); ++_iter984)
+    std::vector<std::string> ::const_iterator _iter992;
+    for (_iter992 = this->part_vals.begin(); _iter992 != this->part_vals.end(); ++_iter992)
     {
-      xfer += oprot->writeString((*_iter984));
+      xfer += oprot->writeString((*_iter992));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12400,10 +12400,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter985;
-    for (_iter985 = (*(this->part_vals)).begin(); _iter985 != (*(this->part_vals)).end(); ++_iter985)
+    std::vector<std::string> ::const_iterator _iter993;
+    for (_iter993 = (*(this->part_vals)).begin(); _iter993 != (*(this->part_vals)).end(); ++_iter993)
     {
-      xfer += oprot->writeString((*_iter985));
+      xfer += oprot->writeString((*_iter993));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12592,17 +12592,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size986;
-            ::apache::thrift::protocol::TType _ktype987;
-            ::apache::thrift::protocol::TType _vtype988;
-            xfer += iprot->readMapBegin(_ktype987, _vtype988, _size986);
-            uint32_t _i990;
-            for (_i990 = 0; _i990 < _size986; ++_i990)
+            uint32_t _size994;
+            ::apache::thrift::protocol::TType _ktype995;
+            ::apache::thrift::protocol::TType _vtype996;
+            xfer += iprot->readMapBegin(_ktype995, _vtype996, _size994);
+            uint32_t _i998;
+            for (_i998 = 0; _i998 < _size994; ++_i998)
             {
-              std::string _key991;
-              xfer += iprot->readString(_key991);
-              std::string& _val992 = this->partitionSpecs[_key991];
-              xfer += iprot->readString(_val992);
+              std::string _key999;
+              xfer += iprot->readString(_key999);
+              std::string& _val1000 = this->partitionSpecs[_key999];
+              xfer += iprot->readString(_val1000);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12663,11 +12663,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter993;
-    for (_iter993 = this->partitionSpecs.begin(); _iter993 != this->partitionSpecs.end(); ++_iter993)
+    std::map<std::string, std::string> ::const_iterator _iter1001;
+    for (_iter1001 = this->partitionSpecs.begin(); _iter1001 != this->partitionSpecs.end(); ++_iter1001)
     {
-      xfer += oprot->writeString(_iter993->first);
-      xfer += oprot->writeString(_iter993->second);
+      xfer += oprot->writeString(_iter1001->first);
+      xfer += oprot->writeString(_iter1001->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12707,11 +12707,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter994;
-    for (_iter994 = (*(this->partitionSpecs)).begin(); _iter994 != (*(this->partitionSpecs)).end(); ++_iter994)
+    std::map<std::string, std::string> ::const_iterator _iter1002;
+    for (_iter1002 = (*(this->partitionSpecs)).begin(); _iter1002 != (*(this->partitionSpecs)).end(); ++_iter1002)
     {
-      xfer += oprot->writeString(_iter994->first);
-      xfer += oprot->writeString(_iter994->second);
+      xfer += oprot->writeString(_iter1002->first);
+      xfer += oprot->writeString(_iter1002->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12956,17 +12956,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size995;
-            ::apache::thrift::protocol::TType _ktype996;
-            ::apache::thrift::protocol::TType _vtype997;
-            xfer += iprot->readMapBegin(_ktype996, _vtype997, _size995);
-            uint32_t _i999;
-            for (_i999 = 0; _i999 < _size995; ++_i999)
+            uint32_t _size1003;
+            ::apache::thrift::protocol::TType _ktype1004;
+            ::apache::thrift::protocol::TType _vtype1005;
+            xfer += iprot->readMapBegin(_ktype1004, _vtype1005, _size1003);
+            uint32_t _i1007;
+            for (_i1007 = 0; _i1007 < _size1003; ++_i1007)
             {
-              std::string _key1000;
-              xfer += iprot->readString(_key1000);
-              std::string& _val1001 = this->partitionSpecs[_key1000];
-              xfer += iprot->readString(_val1001);
+              std::string _key1008;
+              xfer += iprot->readString(_key1008);
+              std::string& _val1009 = this->partitionSpecs[_key1008];
+              xfer += iprot->readString(_val1009);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13027,11 +13027,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter1002;
-    for (_iter1002 = this->partitionSpecs.begin(); _iter1002 != this->partitionSpecs.end(); ++_iter1002)
+    std::map<std::string, std::string> ::const_iterator _iter1010;
+    for (_iter1010 = this->partitionSpecs.begin(); _iter1010 != this->partitionSpecs.end(); ++_iter1010)
     {
-      xfer += oprot->writeString(_iter1002->first);
-      xfer += oprot->writeString(_iter1002->second);
+      xfer += oprot->writeString(_iter1010->first);
+      xfer += oprot->writeString(_iter1010->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13071,11 +13071,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter1003;
-    for (_iter1003 = (*(this->partitionSpecs)).begin(); _iter1003 != (*(this->partitionSpecs)).end(); ++_iter1003)
+    std::map<std::string, std::string> ::const_iterator _iter1011;
+    for (_iter1011 = (*(this->partitionSpecs)).begin(); _iter1011 != (*(this->partitionSpecs)).end(); ++_iter1011)
     {
-      xfer += oprot->writeString(_iter1003->first);
-      xfer += oprot->writeString(_iter1003->second);
+      xfer += oprot->writeString(_iter1011->first);
+      xfer += oprot->writeString(_iter1011->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13132,14 +13132,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1004;
-            ::apache::thrift::protocol::TType _etype1007;
-            xfer += iprot->readListBegin(_etype1007, _size1004);
-            this->success.resize(_size1004);
-            uint32_t _i1008;
-            for (_i1008 = 0; _i1008 < _size1004; ++_i1008)
+            uint32_t _size1012;
+            ::apache::thrift::protocol::TType _etype1015;
+            xfer += iprot->readListBegin(_etype1015, _size1012);
+            this->success.resize(_size1012);
+            uint32_t _i1016;
+            for (_i1016 = 0; _i1016 < _size1012; ++_i1016)
             {
-              xfer += this->success[_i1008].read(iprot);
+              xfer += this->success[_i1016].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13202,10 +13202,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1009;
-      for (_iter1009 = this->success.begin(); _iter1009 != this->success.end(); ++_iter1009)
+      std::vector<Partition> ::const_iterator _iter1017;
+      for (_iter1017 = this->success.begin(); _iter1017 != this->success.end(); ++_iter1017)
       {
-        xfer += (*_iter1009).write(oprot);
+        xfer += (*_iter1017).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13262,14 +13262,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1010;
-            ::apache::thrift::protocol::TType _etype1013;
-            xfer += iprot->readListBegin(_etype1013, _size1010);
-            (*(this->success)).resize(_size1010);
-            uint32_t _i1014;
-            for (_i1014 = 0; _i1014 < _size1010; ++_i1014)
+            uint32_t _size1018;
+            ::apache::thrift::protocol::TType _etype1021;
+            xfer += iprot->readListBegin(_etype1021, _size1018);
+            (*(this->success)).resize(_size1018);
+            uint32_t _i1022;
+            for (_i1022 = 0; _i1022 < _size1018; ++_i1022)
             {
-              xfer += (*(this->success))[_i1014].read(iprot);
+              xfer += (*(this->success))[_i1022].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13368,14 +13368,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1015;
-            ::apache::thrift::protocol::TType _etype1018;
-            xfer += iprot->readListBegin(_etype1018, _size1015);
-            this->part_vals.resize(_size1015);
-            uint32_t _i1019;
-            for (_i1019 = 0; _i1019 < _size1015; ++_i1019)
+            uint32_t _size1023;
+            ::apache::thrift::protocol::TType _etype1026;
+            xfer += iprot->readListBegin(_etype1026, _size1023);
+            this->part_vals.resize(_size1023);
+            uint32_t _i1027;
+            for (_i1027 = 0; _i1027 < _size1023; ++_i1027)
             {
-              xfer += iprot->readString(this->part_vals[_i1019]);
+              xfer += iprot->readString(this->part_vals[_i1027]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13396,14 +13396,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1020;
-            ::apache::thrift::protocol::TType _etype1023;
-            xfer += iprot->readListBegin(_etype1023, _size1020);
-            this->group_names.resize(_size1020);
-            uint32_t _i1024;
-            for (_i1024 = 0; _i1024 < _size1020; ++_i1024)
+            uint32_t _size1028;
+            ::apache::thrift::protocol::TType _etype1031;
+            xfer += iprot->readListBegin(_etype1031, _size1028);
+            this->group_names.resize(_size1028);
+            uint32_t _i1032;
+            for (_i1032 = 0; _i1032 < _size1028; ++_i1032)
             {
-              xfer += iprot->readString(this->group_names[_i1024]);
+              xfer += iprot->readString(this->group_names[_i1032]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13440,10 +13440,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1025;
-    for (_iter1025 = this->part_vals.begin(); _iter1025 != this->part_vals.end(); ++_iter1025)
+    std::vector<std::string> ::const_iterator _iter1033;
+    for (_iter1033 = this->part_vals.begin(); _iter1033 != this->part_vals.end(); ++_iter1033)
     {
-      xfer += oprot->writeString((*_iter1025));
+      xfer += oprot->writeString((*_iter1033));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13456,10 +13456,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1026;
-    for (_iter1026 = this->group_names.begin(); _iter1026 != this->group_names.end(); ++_iter1026)
+    std::vector<std::string> ::const_iterator _iter1034;
+    for (_iter1034 = this->group_names.begin(); _iter1034 != this->group_names.end(); ++_iter1034)
     {
-      xfer += oprot->writeString((*_iter1026));
+      xfer += oprot->writeString((*_iter1034));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13491,10 +13491,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1027;
-    for (_iter1027 = (*(this->part_vals)).begin(); _iter1027 != (*(this->part_vals)).end(); ++_iter1027)
+    std::vector<std::string> ::const_iterator _iter1035;
+    for (_iter1035 = (*(this->part_vals)).begin(); _iter1035 != (*(this->part_vals)).end(); ++_iter1035)
     {
-      xfer += oprot->writeString((*_iter1027));
+      xfer += oprot->writeString((*_iter1035));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13507,10 +13507,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1028;
-    for (_iter1028 = (*(this->group_names)).begin(); _iter1028 != (*(this->group_names)).end(); ++_iter1028)
+    std::vector<std::string> ::const_iterator _iter1036;
+    for (_iter1036 = (*(this->group_names)).begin(); _iter1036 != (*(this->group_names)).end(); ++_iter1036)
     {
-      xfer += oprot->writeString((*_iter1028));
+      xfer += oprot->writeString((*_iter1036));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14069,14 +14069,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1029;
-            ::apache::thrift::protocol::TType _etype1032;
-            xfer += iprot->readListBegin(_etype1032, _size1029);
-            this->success.resize(_size1029);
-            uint32_t _i1033;
-            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
+            uint32_t _size1037;
+            ::apache::thrift::protocol::TType _etype1040;
+            xfer += iprot->readListBegin(_etype1040, _size1037);
+            this->success.resize(_size1037);
+            uint32_t _i1041;
+            for (_i1041 = 0; _i1041 < _size1037; ++_i1041)
             {
-              xfer += this->success[_i1033].read(iprot);
+              xfer += this->success[_i1041].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14123,10 +14123,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1034;
-      for (_iter1034 = this->success.begin(); _iter1034 != this->success.end(); ++_iter1034)
+      std::vector<Partition> ::const_iterator _iter1042;
+      for (_iter1042 = this->success.begin(); _iter1042 != this->success.end(); ++_iter1042)
       {
-        xfer += (*_iter1034).write(oprot);
+        xfer += (*_iter1042).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14175,14 +14175,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1035;
-            ::apache::thrift::protocol::TType _etype1038;
-            xfer += iprot->readListBegin(_etype1038, _size1035);
-            (*(this->success)).resize(_size1035);
-            uint32_t _i1039;
-            for (_i1039 = 0; _i1039 < _size1035; ++_i1039)
+            uint32_t _size1043;
+            ::apache::thrift::protocol::TType _etype1046;
+            xfer += iprot->readListBegin(_etype1046, _size1043);
+            (*(this->success)).resize(_size1043);
+            uint32_t _i1047;
+            for (_i1047 = 0; _i1047 < _size1043; ++_i1047)
             {
-              xfer += (*(this->success))[_i1039].read(iprot);
+              xfer += (*(this->success))[_i1047].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14281,14 +14281,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size1040;
-            ::apache::thrift::protocol::TType _etype1043;
-            xfer += iprot->readListBegin(_etype1043, _size1040);
-            this->group_names.resize(_size1040);
-            uint32_t _i1044;
-            for (_i1044 = 0; _i1044 < _size1040; ++_i1044)
+            uint32_t _size1048;
+            ::apache::thrift::protocol::TType _etype1051;
+            xfer += iprot->readListBegin(_etype1051, _size1048);
+            this->group_names.resize(_size1048);
+            uint32_t _i1052;
+            for (_i1052 = 0; _i1052 < _size1048; ++_i1052)
             {
-              xfer += iprot->readString(this->group_names[_i1044]);
+              xfer += iprot->readString(this->group_names[_i1052]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14333,10 +14333,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter1045;
-    for (_iter1045 = this->group_names.begin(); _iter1045 != this->group_names.end(); ++_iter1045)
+    std::vector<std::string> ::const_iterator _iter1053;
+    for (_iter1053 = this->group_names.begin(); _iter1053 != this->group_names.end(); ++_iter1053)
     {
-      xfer += oprot->writeString((*_iter1045));
+      xfer += oprot->writeString((*_iter1053));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14376,10 +14376,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1046;
-    for (_iter1046 = (*(this->group_names)).begin(); _iter1046 != (*(this->group_names)).end(); ++_iter1046)
+    std::vector<std::string> ::const_iterator _iter1054;
+    for (_iter1054 = (*(this->group_names)).begin(); _iter1054 != (*(this->group_names)).end(); ++_iter1054)
     {
-      xfer += oprot->writeString((*_iter1046));
+      xfer += oprot->writeString((*_iter1054));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14420,14 +14420,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1047;
-            ::apache::thrift::protocol::TType _etype1050;
-            xfer += iprot->readListBegin(_etype1050, _size1047);
-            this->success.resize(_size1047);
-            uint32_t _i1051;
-            for (_i1051 = 0; _i1051 < _size1047; ++_i1051)
+            uint32_t _size1055;
+            ::apache::thrift::protocol::TType _etype1058;
+            xfer += iprot->readListBegin(_etype1058, _size1055);
+            this->success.resize(_size1055);
+            uint32_t _i1059;
+            for (_i1059 = 0; _i1059 < _size1055; ++_i1059)
             {
-              xfer += this->success[_i1051].read(iprot);
+              xfer += this->success[_i1059].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14474,10 +14474,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter1052;
-      for (_iter1052 = this->success.begin(); _iter1052 != this->success.end(); ++_iter1052)
+      std::vector<Partition> ::const_iterator _iter1060;
+      for (_iter1060 = this->success.begin(); _iter1060 != this->success.end(); ++_iter1060)
       {
-        xfer += (*_iter1052).write(oprot);
+        xfer += (*_iter1060).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14526,14 +14526,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1053;
-            ::apache::thrift::protocol::TType _etype1056;
-            xfer += iprot->readListBegin(_etype1056, _size1053);
-            (*(this->success)).resize(_size1053);
-            uint32_t _i1057;
-            for (_i1057 = 0; _i1057 < _size1053; ++_i1057)
+            uint32_t _size1061;
+            ::apache::thrift::protocol::TType _etype1064;
+            xfer += iprot->readListBegin(_etype1064, _size1061);
+            (*(this->success)).resize(_size1061);
+            uint32_t _i1065;
+            for (_i1065 = 0; _i1065 < _size1061; ++_i1065)
             {
-              xfer += (*(this->success))[_i1057].read(iprot);
+              xfer += (*(this->success))[_i1065].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14711,14 +14711,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1058;
-            ::apache::thrift::protocol::TType _etype1061;
-            xfer += iprot->readListBegin(_etype1061, _size1058);
-            this->success.resize(_size1058);
-            uint32_t _i1062;
-            for (_i1062 = 0; _i1062 < _size1058; ++_i1062)
+            uint32_t _size1066;
+            ::apache::thrift::protocol::TType _etype1069;
+            xfer += iprot->readListBegin(_etype1069, _size1066);
+            this->success.resize(_size1066);
+            uint32_t _i1070;
+            for (_i1070 = 0; _i1070 < _size1066; ++_i1070)
             {
-              xfer += this->success[_i1062].read(iprot);
+              xfer += this->success[_i1070].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14765,10 +14765,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter1063;
-      for (_iter1063 = this->success.begin(); _iter1063 != this->success.end(); ++_iter1063)
+      std::vector<PartitionSpec> ::const_iterator _iter1071;
+      for (_iter1071 = this->success.begin(); _iter1071 != this->success.end(); ++_iter1071)
       {
-        xfer += (*_iter1063).write(oprot);
+        xfer += (*_iter1071).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14817,14 +14817,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1064;
-            ::apache::thrift::protocol::TType _etype1067;
-            xfer += iprot->readListBegin(_etype1067, _size1064);
-            (*(this->success)).resize(_size1064);
-            uint32_t _i1068;
-            for (_i1068 = 0; _i1068 < _size1064; ++_i1068)
+            uint32_t _size1072;
+            ::apache::thrift::protocol::TType _etype1075;
+            xfer += iprot->readListBegin(_etype1075, _size1072);
+            (*(this->success)).resize(_size1072);
+            uint32_t _i1076;
+            for (_i1076 = 0; _i1076 < _size1072; ++_i1076)
             {
-              xfer += (*(this->success))[_i1068].read(iprot);
+              xfer += (*(this->success))[_i1076].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15002,14 +15002,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1069;
-            ::apache::thrift::protocol::TType _etype1072;
-            xfer += iprot->readListBegin(_etype1072, _size1069);
-            this->success.resize(_size1069);
-            uint32_t _i1073;
-            for (_i1073 = 0; _i1073 < _size1069; ++_i1073)
+            uint32_t _size1077;
+            ::apache::thrift::protocol::TType _etype1080;
+            xfer += iprot->readListBegin(_etype1080, _size1077);
+            this->success.resize(_size1077);
+            uint32_t _i1081;
+            for (_i1081 = 0; _i1081 < _size1077; ++_i1081)
             {
-              xfer += iprot->readString(this->success[_i1073]);
+              xfer += iprot->readString(this->su

<TRUNCATED>

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

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index 5aef35c..f505208 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -14373,6 +14373,10 @@ class CompactionRequest {
    * @var string
    */
   public $runas = null;
+  /**
+   * @var array
+   */
+  public $properties = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -14397,6 +14401,18 @@ class CompactionRequest {
           'var' => 'runas',
           'type' => TType::STRING,
           ),
+        6 => array(
+          'var' => 'properties',
+          'type' => TType::MAP,
+          'ktype' => TType::STRING,
+          'vtype' => TType::STRING,
+          'key' => array(
+            'type' => TType::STRING,
+          ),
+          'val' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -14415,6 +14431,9 @@ class CompactionRequest {
       if (isset($vals['runas'])) {
         $this->runas = $vals['runas'];
       }
+      if (isset($vals['properties'])) {
+        $this->properties = $vals['properties'];
+      }
     }
   }
 
@@ -14472,6 +14491,26 @@ class CompactionRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::MAP) {
+            $this->properties = array();
+            $_size465 = 0;
+            $_ktype466 = 0;
+            $_vtype467 = 0;
+            $xfer += $input->readMapBegin($_ktype466, $_vtype467, $_size465);
+            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            {
+              $key470 = '';
+              $val471 = '';
+              $xfer += $input->readString($key470);
+              $xfer += $input->readString($val471);
+              $this->properties[$key470] = $val471;
+            }
+            $xfer += $input->readMapEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -14510,6 +14549,24 @@ class CompactionRequest {
       $xfer += $output->writeString($this->runas);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->properties !== null) {
+      if (!is_array($this->properties)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('properties', TType::MAP, 6);
+      {
+        $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
+        {
+          foreach ($this->properties as $kiter472 => $viter473)
+          {
+            $xfer += $output->writeString($kiter472);
+            $xfer += $output->writeString($viter473);
+          }
+        }
+        $output->writeMapEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -14946,15 +15003,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size465 = 0;
-            $_etype468 = 0;
-            $xfer += $input->readListBegin($_etype468, $_size465);
-            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            $_size474 = 0;
+            $_etype477 = 0;
+            $xfer += $input->readListBegin($_etype477, $_size474);
+            for ($_i478 = 0; $_i478 < $_size474; ++$_i478)
             {
-              $elem470 = null;
-              $elem470 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem470->read($input);
-              $this->compacts []= $elem470;
+              $elem479 = null;
+              $elem479 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem479->read($input);
+              $this->compacts []= $elem479;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14982,9 +15039,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter471)
+          foreach ($this->compacts as $iter480)
           {
-            $xfer += $iter471->write($output);
+            $xfer += $iter480->write($output);
           }
         }
         $output->writeListEnd();
@@ -15113,14 +15170,14 @@ class AddDynamicPartitions {
         case 4:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size472 = 0;
-            $_etype475 = 0;
-            $xfer += $input->readListBegin($_etype475, $_size472);
-            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
+            $_size481 = 0;
+            $_etype484 = 0;
+            $xfer += $input->readListBegin($_etype484, $_size481);
+            for ($_i485 = 0; $_i485 < $_size481; ++$_i485)
             {
-              $elem477 = null;
-              $xfer += $input->readString($elem477);
-              $this->partitionnames []= $elem477;
+              $elem486 = null;
+              $xfer += $input->readString($elem486);
+              $this->partitionnames []= $elem486;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15170,9 +15227,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter478)
+          foreach ($this->partitionnames as $iter487)
           {
-            $xfer += $output->writeString($iter478);
+            $xfer += $output->writeString($iter487);
           }
         }
         $output->writeListEnd();
@@ -15530,15 +15587,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size479 = 0;
-            $_etype482 = 0;
-            $xfer += $input->readListBegin($_etype482, $_size479);
-            for ($_i483 = 0; $_i483 < $_size479; ++$_i483)
+            $_size488 = 0;
+            $_etype491 = 0;
+            $xfer += $input->readListBegin($_etype491, $_size488);
+            for ($_i492 = 0; $_i492 < $_size488; ++$_i492)
             {
-              $elem484 = null;
-              $elem484 = new \metastore\NotificationEvent();
-              $xfer += $elem484->read($input);
-              $this->events []= $elem484;
+              $elem493 = null;
+              $elem493 = new \metastore\NotificationEvent();
+              $xfer += $elem493->read($input);
+              $this->events []= $elem493;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15566,9 +15623,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter485)
+          foreach ($this->events as $iter494)
           {
-            $xfer += $iter485->write($output);
+            $xfer += $iter494->write($output);
           }
         }
         $output->writeListEnd();
@@ -15707,14 +15764,14 @@ class InsertEventRequestData {
         case 1:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size486 = 0;
-            $_etype489 = 0;
-            $xfer += $input->readListBegin($_etype489, $_size486);
-            for ($_i490 = 0; $_i490 < $_size486; ++$_i490)
+            $_size495 = 0;
+            $_etype498 = 0;
+            $xfer += $input->readListBegin($_etype498, $_size495);
+            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
             {
-              $elem491 = null;
-              $xfer += $input->readString($elem491);
-              $this->filesAdded []= $elem491;
+              $elem500 = null;
+              $xfer += $input->readString($elem500);
+              $this->filesAdded []= $elem500;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15742,9 +15799,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter492)
+          foreach ($this->filesAdded as $iter501)
           {
-            $xfer += $output->writeString($iter492);
+            $xfer += $output->writeString($iter501);
           }
         }
         $output->writeListEnd();
@@ -15962,14 +16019,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size493 = 0;
-            $_etype496 = 0;
-            $xfer += $input->readListBegin($_etype496, $_size493);
-            for ($_i497 = 0; $_i497 < $_size493; ++$_i497)
+            $_size502 = 0;
+            $_etype505 = 0;
+            $xfer += $input->readListBegin($_etype505, $_size502);
+            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
             {
-              $elem498 = null;
-              $xfer += $input->readString($elem498);
-              $this->partitionVals []= $elem498;
+              $elem507 = null;
+              $xfer += $input->readString($elem507);
+              $this->partitionVals []= $elem507;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16020,9 +16077,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter499)
+          foreach ($this->partitionVals as $iter508)
           {
-            $xfer += $output->writeString($iter499);
+            $xfer += $output->writeString($iter508);
           }
         }
         $output->writeListEnd();
@@ -16250,18 +16307,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size500 = 0;
-            $_ktype501 = 0;
-            $_vtype502 = 0;
-            $xfer += $input->readMapBegin($_ktype501, $_vtype502, $_size500);
-            for ($_i504 = 0; $_i504 < $_size500; ++$_i504)
+            $_size509 = 0;
+            $_ktype510 = 0;
+            $_vtype511 = 0;
+            $xfer += $input->readMapBegin($_ktype510, $_vtype511, $_size509);
+            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
             {
-              $key505 = 0;
-              $val506 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key505);
-              $val506 = new \metastore\MetadataPpdResult();
-              $xfer += $val506->read($input);
-              $this->metadata[$key505] = $val506;
+              $key514 = 0;
+              $val515 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key514);
+              $val515 = new \metastore\MetadataPpdResult();
+              $xfer += $val515->read($input);
+              $this->metadata[$key514] = $val515;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16296,10 +16353,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter507 => $viter508)
+          foreach ($this->metadata as $kiter516 => $viter517)
           {
-            $xfer += $output->writeI64($kiter507);
-            $xfer += $viter508->write($output);
+            $xfer += $output->writeI64($kiter516);
+            $xfer += $viter517->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16401,14 +16458,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size509 = 0;
-            $_etype512 = 0;
-            $xfer += $input->readListBegin($_etype512, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size518 = 0;
+            $_etype521 = 0;
+            $xfer += $input->readListBegin($_etype521, $_size518);
+            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
             {
-              $elem514 = null;
-              $xfer += $input->readI64($elem514);
-              $this->fileIds []= $elem514;
+              $elem523 = null;
+              $xfer += $input->readI64($elem523);
+              $this->fileIds []= $elem523;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16457,9 +16514,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter515)
+          foreach ($this->fileIds as $iter524)
           {
-            $xfer += $output->writeI64($iter515);
+            $xfer += $output->writeI64($iter524);
           }
         }
         $output->writeListEnd();
@@ -16553,17 +16610,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size516 = 0;
-            $_ktype517 = 0;
-            $_vtype518 = 0;
-            $xfer += $input->readMapBegin($_ktype517, $_vtype518, $_size516);
-            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
+            $_size525 = 0;
+            $_ktype526 = 0;
+            $_vtype527 = 0;
+            $xfer += $input->readMapBegin($_ktype526, $_vtype527, $_size525);
+            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
             {
-              $key521 = 0;
-              $val522 = '';
-              $xfer += $input->readI64($key521);
-              $xfer += $input->readString($val522);
-              $this->metadata[$key521] = $val522;
+              $key530 = 0;
+              $val531 = '';
+              $xfer += $input->readI64($key530);
+              $xfer += $input->readString($val531);
+              $this->metadata[$key530] = $val531;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16598,10 +16655,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter523 => $viter524)
+          foreach ($this->metadata as $kiter532 => $viter533)
           {
-            $xfer += $output->writeI64($kiter523);
-            $xfer += $output->writeString($viter524);
+            $xfer += $output->writeI64($kiter532);
+            $xfer += $output->writeString($viter533);
           }
         }
         $output->writeMapEnd();
@@ -16670,14 +16727,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size525 = 0;
-            $_etype528 = 0;
-            $xfer += $input->readListBegin($_etype528, $_size525);
-            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
+            $_size534 = 0;
+            $_etype537 = 0;
+            $xfer += $input->readListBegin($_etype537, $_size534);
+            for ($_i538 = 0; $_i538 < $_size534; ++$_i538)
             {
-              $elem530 = null;
-              $xfer += $input->readI64($elem530);
-              $this->fileIds []= $elem530;
+              $elem539 = null;
+              $xfer += $input->readI64($elem539);
+              $this->fileIds []= $elem539;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16705,9 +16762,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter531)
+          foreach ($this->fileIds as $iter540)
           {
-            $xfer += $output->writeI64($iter531);
+            $xfer += $output->writeI64($iter540);
           }
         }
         $output->writeListEnd();
@@ -16847,14 +16904,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size532 = 0;
-            $_etype535 = 0;
-            $xfer += $input->readListBegin($_etype535, $_size532);
-            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
+            $_size541 = 0;
+            $_etype544 = 0;
+            $xfer += $input->readListBegin($_etype544, $_size541);
+            for ($_i545 = 0; $_i545 < $_size541; ++$_i545)
             {
-              $elem537 = null;
-              $xfer += $input->readI64($elem537);
-              $this->fileIds []= $elem537;
+              $elem546 = null;
+              $xfer += $input->readI64($elem546);
+              $this->fileIds []= $elem546;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16864,14 +16921,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size538 = 0;
-            $_etype541 = 0;
-            $xfer += $input->readListBegin($_etype541, $_size538);
-            for ($_i542 = 0; $_i542 < $_size538; ++$_i542)
+            $_size547 = 0;
+            $_etype550 = 0;
+            $xfer += $input->readListBegin($_etype550, $_size547);
+            for ($_i551 = 0; $_i551 < $_size547; ++$_i551)
             {
-              $elem543 = null;
-              $xfer += $input->readString($elem543);
-              $this->metadata []= $elem543;
+              $elem552 = null;
+              $xfer += $input->readString($elem552);
+              $this->metadata []= $elem552;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16906,9 +16963,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter544)
+          foreach ($this->fileIds as $iter553)
           {
-            $xfer += $output->writeI64($iter544);
+            $xfer += $output->writeI64($iter553);
           }
         }
         $output->writeListEnd();
@@ -16923,9 +16980,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter545)
+          foreach ($this->metadata as $iter554)
           {
-            $xfer += $output->writeString($iter545);
+            $xfer += $output->writeString($iter554);
           }
         }
         $output->writeListEnd();
@@ -17044,14 +17101,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size546 = 0;
-            $_etype549 = 0;
-            $xfer += $input->readListBegin($_etype549, $_size546);
-            for ($_i550 = 0; $_i550 < $_size546; ++$_i550)
+            $_size555 = 0;
+            $_etype558 = 0;
+            $xfer += $input->readListBegin($_etype558, $_size555);
+            for ($_i559 = 0; $_i559 < $_size555; ++$_i559)
             {
-              $elem551 = null;
-              $xfer += $input->readI64($elem551);
-              $this->fileIds []= $elem551;
+              $elem560 = null;
+              $xfer += $input->readI64($elem560);
+              $this->fileIds []= $elem560;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17079,9 +17136,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter552)
+          foreach ($this->fileIds as $iter561)
           {
-            $xfer += $output->writeI64($iter552);
+            $xfer += $output->writeI64($iter561);
           }
         }
         $output->writeListEnd();
@@ -17365,15 +17422,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size553 = 0;
-            $_etype556 = 0;
-            $xfer += $input->readListBegin($_etype556, $_size553);
-            for ($_i557 = 0; $_i557 < $_size553; ++$_i557)
+            $_size562 = 0;
+            $_etype565 = 0;
+            $xfer += $input->readListBegin($_etype565, $_size562);
+            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
             {
-              $elem558 = null;
-              $elem558 = new \metastore\Function();
-              $xfer += $elem558->read($input);
-              $this->functions []= $elem558;
+              $elem567 = null;
+              $elem567 = new \metastore\Function();
+              $xfer += $elem567->read($input);
+              $this->functions []= $elem567;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17401,9 +17458,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter559)
+          foreach ($this->functions as $iter568)
           {
-            $xfer += $iter559->write($output);
+            $xfer += $iter568->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 0d70bb2..57a748a 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -11298,10 +11298,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype560, _size557) = iprot.readListBegin()
-          for _i561 in xrange(_size557):
-            _elem562 = iprot.readString()
-            self.success.append(_elem562)
+          (_etype569, _size566) = iprot.readListBegin()
+          for _i570 in xrange(_size566):
+            _elem571 = iprot.readString()
+            self.success.append(_elem571)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11324,8 +11324,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 iter563 in self.success:
-        oprot.writeString(iter563)
+      for iter572 in self.success:
+        oprot.writeString(iter572)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11430,10 +11430,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype567, _size564) = iprot.readListBegin()
-          for _i568 in xrange(_size564):
-            _elem569 = iprot.readString()
-            self.success.append(_elem569)
+          (_etype576, _size573) = iprot.readListBegin()
+          for _i577 in xrange(_size573):
+            _elem578 = iprot.readString()
+            self.success.append(_elem578)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11456,8 +11456,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 iter570 in self.success:
-        oprot.writeString(iter570)
+      for iter579 in self.success:
+        oprot.writeString(iter579)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12227,12 +12227,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype572, _vtype573, _size571 ) = iprot.readMapBegin()
-          for _i575 in xrange(_size571):
-            _key576 = iprot.readString()
-            _val577 = Type()
-            _val577.read(iprot)
-            self.success[_key576] = _val577
+          (_ktype581, _vtype582, _size580 ) = iprot.readMapBegin()
+          for _i584 in xrange(_size580):
+            _key585 = iprot.readString()
+            _val586 = Type()
+            _val586.read(iprot)
+            self.success[_key585] = _val586
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -12255,9 +12255,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 kiter578,viter579 in self.success.items():
-        oprot.writeString(kiter578)
-        viter579.write(oprot)
+      for kiter587,viter588 in self.success.items():
+        oprot.writeString(kiter587)
+        viter588.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -12400,11 +12400,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype583, _size580) = iprot.readListBegin()
-          for _i584 in xrange(_size580):
-            _elem585 = FieldSchema()
-            _elem585.read(iprot)
-            self.success.append(_elem585)
+          (_etype592, _size589) = iprot.readListBegin()
+          for _i593 in xrange(_size589):
+            _elem594 = FieldSchema()
+            _elem594.read(iprot)
+            self.success.append(_elem594)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12439,8 +12439,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 iter586 in self.success:
-        iter586.write(oprot)
+      for iter595 in self.success:
+        iter595.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12607,11 +12607,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = FieldSchema()
-            _elem592.read(iprot)
-            self.success.append(_elem592)
+          (_etype599, _size596) = iprot.readListBegin()
+          for _i600 in xrange(_size596):
+            _elem601 = FieldSchema()
+            _elem601.read(iprot)
+            self.success.append(_elem601)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12646,8 +12646,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 iter593 in self.success:
-        iter593.write(oprot)
+      for iter602 in self.success:
+        iter602.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12800,11 +12800,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = FieldSchema()
-            _elem599.read(iprot)
-            self.success.append(_elem599)
+          (_etype606, _size603) = iprot.readListBegin()
+          for _i607 in xrange(_size603):
+            _elem608 = FieldSchema()
+            _elem608.read(iprot)
+            self.success.append(_elem608)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12839,8 +12839,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 iter600 in self.success:
-        iter600.write(oprot)
+      for iter609 in self.success:
+        iter609.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13007,11 +13007,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype604, _size601) = iprot.readListBegin()
-          for _i605 in xrange(_size601):
-            _elem606 = FieldSchema()
-            _elem606.read(iprot)
-            self.success.append(_elem606)
+          (_etype613, _size610) = iprot.readListBegin()
+          for _i614 in xrange(_size610):
+            _elem615 = FieldSchema()
+            _elem615.read(iprot)
+            self.success.append(_elem615)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13046,8 +13046,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 iter607 in self.success:
-        iter607.write(oprot)
+      for iter616 in self.success:
+        iter616.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13488,22 +13488,22 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype611, _size608) = iprot.readListBegin()
-          for _i612 in xrange(_size608):
-            _elem613 = SQLPrimaryKey()
-            _elem613.read(iprot)
-            self.primaryKeys.append(_elem613)
+          (_etype620, _size617) = iprot.readListBegin()
+          for _i621 in xrange(_size617):
+            _elem622 = SQLPrimaryKey()
+            _elem622.read(iprot)
+            self.primaryKeys.append(_elem622)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype617, _size614) = iprot.readListBegin()
-          for _i618 in xrange(_size614):
-            _elem619 = SQLForeignKey()
-            _elem619.read(iprot)
-            self.foreignKeys.append(_elem619)
+          (_etype626, _size623) = iprot.readListBegin()
+          for _i627 in xrange(_size623):
+            _elem628 = SQLForeignKey()
+            _elem628.read(iprot)
+            self.foreignKeys.append(_elem628)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13524,15 +13524,15 @@ 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 iter620 in self.primaryKeys:
-        iter620.write(oprot)
+      for iter629 in self.primaryKeys:
+        iter629.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 iter621 in self.foreignKeys:
-        iter621.write(oprot)
+      for iter630 in self.foreignKeys:
+        iter630.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14568,10 +14568,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype625, _size622) = iprot.readListBegin()
-          for _i626 in xrange(_size622):
-            _elem627 = iprot.readString()
-            self.success.append(_elem627)
+          (_etype634, _size631) = iprot.readListBegin()
+          for _i635 in xrange(_size631):
+            _elem636 = iprot.readString()
+            self.success.append(_elem636)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14594,8 +14594,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 iter628 in self.success:
-        oprot.writeString(iter628)
+      for iter637 in self.success:
+        oprot.writeString(iter637)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14668,10 +14668,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = iprot.readString()
-            self.tbl_types.append(_elem634)
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = iprot.readString()
+            self.tbl_types.append(_elem643)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14696,8 +14696,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 iter635 in self.tbl_types:
-        oprot.writeString(iter635)
+      for iter644 in self.tbl_types:
+        oprot.writeString(iter644)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14753,11 +14753,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype639, _size636) = iprot.readListBegin()
-          for _i640 in xrange(_size636):
-            _elem641 = TableMeta()
-            _elem641.read(iprot)
-            self.success.append(_elem641)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = TableMeta()
+            _elem650.read(iprot)
+            self.success.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14780,8 +14780,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 iter642 in self.success:
-        iter642.write(oprot)
+      for iter651 in self.success:
+        iter651.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14905,10 +14905,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.success.append(_elem648)
+          (_etype655, _size652) = iprot.readListBegin()
+          for _i656 in xrange(_size652):
+            _elem657 = iprot.readString()
+            self.success.append(_elem657)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14931,8 +14931,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 iter649 in self.success:
-        oprot.writeString(iter649)
+      for iter658 in self.success:
+        oprot.writeString(iter658)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15168,10 +15168,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = iprot.readString()
-            self.tbl_names.append(_elem655)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = iprot.readString()
+            self.tbl_names.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15192,8 +15192,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 iter656 in self.tbl_names:
-        oprot.writeString(iter656)
+      for iter665 in self.tbl_names:
+        oprot.writeString(iter665)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15254,11 +15254,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype660, _size657) = iprot.readListBegin()
-          for _i661 in xrange(_size657):
-            _elem662 = Table()
-            _elem662.read(iprot)
-            self.success.append(_elem662)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = Table()
+            _elem671.read(iprot)
+            self.success.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15293,8 +15293,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 iter663 in self.success:
-        iter663.write(oprot)
+      for iter672 in self.success:
+        iter672.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15460,10 +15460,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype667, _size664) = iprot.readListBegin()
-          for _i668 in xrange(_size664):
-            _elem669 = iprot.readString()
-            self.success.append(_elem669)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = iprot.readString()
+            self.success.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15498,8 +15498,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 iter670 in self.success:
-        oprot.writeString(iter670)
+      for iter679 in self.success:
+        oprot.writeString(iter679)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16469,11 +16469,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype674, _size671) = iprot.readListBegin()
-          for _i675 in xrange(_size671):
-            _elem676 = Partition()
-            _elem676.read(iprot)
-            self.new_parts.append(_elem676)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = Partition()
+            _elem685.read(iprot)
+            self.new_parts.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16490,8 +16490,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 iter677 in self.new_parts:
-        iter677.write(oprot)
+      for iter686 in self.new_parts:
+        iter686.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16649,11 +16649,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype681, _size678) = iprot.readListBegin()
-          for _i682 in xrange(_size678):
-            _elem683 = PartitionSpec()
-            _elem683.read(iprot)
-            self.new_parts.append(_elem683)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = PartitionSpec()
+            _elem692.read(iprot)
+            self.new_parts.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16670,8 +16670,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 iter684 in self.new_parts:
-        iter684.write(oprot)
+      for iter693 in self.new_parts:
+        iter693.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16845,10 +16845,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype688, _size685) = iprot.readListBegin()
-          for _i689 in xrange(_size685):
-            _elem690 = iprot.readString()
-            self.part_vals.append(_elem690)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = iprot.readString()
+            self.part_vals.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16873,8 +16873,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 iter691 in self.part_vals:
-        oprot.writeString(iter691)
+      for iter700 in self.part_vals:
+        oprot.writeString(iter700)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17227,10 +17227,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype695, _size692) = iprot.readListBegin()
-          for _i696 in xrange(_size692):
-            _elem697 = iprot.readString()
-            self.part_vals.append(_elem697)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = iprot.readString()
+            self.part_vals.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17261,8 +17261,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 iter698 in self.part_vals:
-        oprot.writeString(iter698)
+      for iter707 in self.part_vals:
+        oprot.writeString(iter707)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -17857,10 +17857,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype702, _size699) = iprot.readListBegin()
-          for _i703 in xrange(_size699):
-            _elem704 = iprot.readString()
-            self.part_vals.append(_elem704)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.part_vals.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17890,8 +17890,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 iter705 in self.part_vals:
-        oprot.writeString(iter705)
+      for iter714 in self.part_vals:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18064,10 +18064,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype709, _size706) = iprot.readListBegin()
-          for _i710 in xrange(_size706):
-            _elem711 = iprot.readString()
-            self.part_vals.append(_elem711)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = iprot.readString()
+            self.part_vals.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18103,8 +18103,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 iter712 in self.part_vals:
-        oprot.writeString(iter712)
+      for iter721 in self.part_vals:
+        oprot.writeString(iter721)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18841,10 +18841,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype716, _size713) = iprot.readListBegin()
-          for _i717 in xrange(_size713):
-            _elem718 = iprot.readString()
-            self.part_vals.append(_elem718)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = iprot.readString()
+            self.part_vals.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18869,8 +18869,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 iter719 in self.part_vals:
-        oprot.writeString(iter719)
+      for iter728 in self.part_vals:
+        oprot.writeString(iter728)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19029,11 +19029,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype721, _vtype722, _size720 ) = iprot.readMapBegin()
-          for _i724 in xrange(_size720):
-            _key725 = iprot.readString()
-            _val726 = iprot.readString()
-            self.partitionSpecs[_key725] = _val726
+          (_ktype730, _vtype731, _size729 ) = iprot.readMapBegin()
+          for _i733 in xrange(_size729):
+            _key734 = iprot.readString()
+            _val735 = iprot.readString()
+            self.partitionSpecs[_key734] = _val735
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19070,9 +19070,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 kiter727,viter728 in self.partitionSpecs.items():
-        oprot.writeString(kiter727)
-        oprot.writeString(viter728)
+      for kiter736,viter737 in self.partitionSpecs.items():
+        oprot.writeString(kiter736)
+        oprot.writeString(viter737)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19277,11 +19277,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype730, _vtype731, _size729 ) = iprot.readMapBegin()
-          for _i733 in xrange(_size729):
-            _key734 = iprot.readString()
-            _val735 = iprot.readString()
-            self.partitionSpecs[_key734] = _val735
+          (_ktype739, _vtype740, _size738 ) = iprot.readMapBegin()
+          for _i742 in xrange(_size738):
+            _key743 = iprot.readString()
+            _val744 = iprot.readString()
+            self.partitionSpecs[_key743] = _val744
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19318,9 +19318,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 kiter736,viter737 in self.partitionSpecs.items():
-        oprot.writeString(kiter736)
-        oprot.writeString(viter737)
+      for kiter745,viter746 in self.partitionSpecs.items():
+        oprot.writeString(kiter745)
+        oprot.writeString(viter746)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19403,11 +19403,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype741, _size738) = iprot.readListBegin()
-          for _i742 in xrange(_size738):
-            _elem743 = Partition()
-            _elem743.read(iprot)
-            self.success.append(_elem743)
+          (_etype750, _size747) = iprot.readListBegin()
+          for _i751 in xrange(_size747):
+            _elem752 = Partition()
+            _elem752.read(iprot)
+            self.success.append(_elem752)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19448,8 +19448,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 iter744 in self.success:
-        iter744.write(oprot)
+      for iter753 in self.success:
+        iter753.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19543,10 +19543,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype748, _size745) = iprot.readListBegin()
-          for _i749 in xrange(_size745):
-            _elem750 = iprot.readString()
-            self.part_vals.append(_elem750)
+          (_etype757, _size754) = iprot.readListBegin()
+          for _i758 in xrange(_size754):
+            _elem759 = iprot.readString()
+            self.part_vals.append(_elem759)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19558,10 +19558,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype754, _size751) = iprot.readListBegin()
-          for _i755 in xrange(_size751):
-            _elem756 = iprot.readString()
-            self.group_names.append(_elem756)
+          (_etype763, _size760) = iprot.readListBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = iprot.readString()
+            self.group_names.append(_elem765)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19586,8 +19586,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 iter757 in self.part_vals:
-        oprot.writeString(iter757)
+      for iter766 in self.part_vals:
+        oprot.writeString(iter766)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -19597,8 +19597,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 iter758 in self.group_names:
-        oprot.writeString(iter758)
+      for iter767 in self.group_names:
+        oprot.writeString(iter767)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20027,11 +20027,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype762, _size759) = iprot.readListBegin()
-          for _i763 in xrange(_size759):
-            _elem764 = Partition()
-            _elem764.read(iprot)
-            self.success.append(_elem764)
+          (_etype771, _size768) = iprot.readListBegin()
+          for _i772 in xrange(_size768):
+            _elem773 = Partition()
+            _elem773.read(iprot)
+            self.success.append(_elem773)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20060,8 +20060,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 iter765 in self.success:
-        iter765.write(oprot)
+      for iter774 in self.success:
+        iter774.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20155,10 +20155,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype769, _size766) = iprot.readListBegin()
-          for _i770 in xrange(_size766):
-            _elem771 = iprot.readString()
-            self.group_names.append(_elem771)
+          (_etype778, _size775) = iprot.readListBegin()
+          for _i779 in xrange(_size775):
+            _elem780 = iprot.readString()
+            self.group_names.append(_elem780)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20191,8 +20191,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 iter772 in self.group_names:
-        oprot.writeString(iter772)
+      for iter781 in self.group_names:
+        oprot.writeString(iter781)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20253,11 +20253,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype776, _size773) = iprot.readListBegin()
-          for _i777 in xrange(_size773):
-            _elem778 = Partition()
-            _elem778.read(iprot)
-            self.success.append(_elem778)
+          (_etype785, _size782) = iprot.readListBegin()
+          for _i786 in xrange(_size782):
+            _elem787 = Partition()
+            _elem787.read(iprot)
+            self.success.append(_elem787)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20286,8 +20286,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 iter779 in self.success:
-        iter779.write(oprot)
+      for iter788 in self.success:
+        iter788.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20445,11 +20445,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype783, _size780) = iprot.readListBegin()
-          for _i784 in xrange(_size780):
-            _elem785 = PartitionSpec()
-            _elem785.read(iprot)
-            self.success.append(_elem785)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = PartitionSpec()
+            _elem794.read(iprot)
+            self.success.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20478,8 +20478,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 iter786 in self.success:
-        iter786.write(oprot)
+      for iter795 in self.success:
+        iter795.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20634,10 +20634,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = iprot.readString()
-            self.success.append(_elem792)
+          (_etype799, _size796) = iprot.readListBegin()
+          for _i800 in xrange(_size796):
+            _elem801 = iprot.readString()
+            self.success.append(_elem801)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20660,8 +20660,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 iter793 in self.success:
-        oprot.writeString(iter793)
+      for iter802 in self.success:
+        oprot.writeString(iter802)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -20737,10 +20737,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype797, _size794) = iprot.readListBegin()
-          for _i798 in xrange(_size794):
-            _elem799 = iprot.readString()
-            self.part_vals.append(_elem799)
+          (_etype806, _size803) = iprot.readListBegin()
+          for _i807 in xrange(_size803):
+            _elem808 = iprot.readString()
+            self.part_vals.append(_elem808)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20770,8 +20770,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 iter800 in self.part_vals:
-        oprot.writeString(iter800)
+      for iter809 in self.part_vals:
+        oprot.writeString(iter809)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -20835,11 +20835,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype804, _size801) = iprot.readListBegin()
-          for _i805 in xrange(_size801):
-            _elem806 = Partition()
-            _elem806.read(iprot)
-            self.success.append(_elem806)
+          (_etype813, _size810) = iprot.readListBegin()
+          for _i814 in xrange(_size810):
+            _elem815 = Partition()
+            _elem815.read(iprot)
+            self.success.append(_elem815)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20868,8 +20868,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 iter807 in self.success:
-        iter807.write(oprot)
+      for iter816 in self.success:
+        iter816.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20956,10 +20956,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype811, _size808) = iprot.readListBegin()
-          for _i812 in xrange(_size808):
-            _elem813 = iprot.readString()
-            self.part_vals.append(_elem813)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = iprot.readString()
+            self.part_vals.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20976,10 +20976,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype817, _size814) = iprot.readListBegin()
-          for _i818 in xrange(_size814):
-            _elem819 = iprot.readString()
-            self.group_names.append(_elem819)
+          (_etype826, _size823) = iprot.readListBegin()
+          for _i827 in xrange(_size823):
+            _elem828 = iprot.readString()
+            self.group_names.append(_elem828)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21004,8 +21004,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 iter820 in self.part_vals:
-        oprot.writeString(iter820)
+      for iter829 in self.part_vals:
+        oprot.writeString(iter829)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21019,8 +21019,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 iter821 in self.group_names:
-        oprot.writeString(iter821)
+      for iter830 in self.group_names:
+        oprot.writeString(iter830)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21082,11 +21082,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype825, _size822) = iprot.readListBegin()
-          for _i826 in xrange(_size822):
-            _elem827 = Partition()
-            _elem827.read(iprot)
-            self.success.append(_elem827)
+          (_etype834, _size831) = iprot.readListBegin()
+          for _i835 in xrange(_size831):
+            _elem836 = Partition()
+            _elem836.read(iprot)
+            self.success.append(_elem836)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21115,8 +21115,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 iter828 in self.success:
-        iter828.write(oprot)
+      for iter837 in self.success:
+        iter837.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21197,10 +21197,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = iprot.readString()
-            self.part_vals.append(_elem834)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = iprot.readString()
+            self.part_vals.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21230,8 +21230,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 iter835 in self.part_vals:
-        oprot.writeString(iter835)
+      for iter844 in self.part_vals:
+        oprot.writeString(iter844)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21295,10 +21295,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype839, _size836) = iprot.readListBegin()
-          for _i840 in xrange(_size836):
-            _elem841 = iprot.readString()
-            self.success.append(_elem841)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = iprot.readString()
+            self.success.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21327,8 +21327,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 iter842 in self.success:
-        oprot.writeString(iter842)
+      for iter851 in self.success:
+        oprot.writeString(iter851)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21499,11 +21499,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype846, _size843) = iprot.readListBegin()
-          for _i847 in xrange(_size843):
-            _elem848 = Partition()
-            _elem848.read(iprot)
-            self.success.append(_elem848)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = Partition()
+            _elem857.read(iprot)
+            self.success.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21532,8 +21532,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 iter849 in self.success:
-        iter849.write(oprot)
+      for iter858 in self.success:
+        iter858.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21704,11 +21704,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype853, _size850) = iprot.readListBegin()
-          for _i854 in xrange(_size850):
-            _elem855 = PartitionSpec()
-            _elem855.read(iprot)
-            self.success.append(_elem855)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = PartitionSpec()
+            _elem864.read(iprot)
+            self.success.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21737,8 +21737,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 iter856 in self.success:
-        iter856.write(oprot)
+      for iter865 in self.success:
+        iter865.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22158,10 +22158,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype860, _size857) = iprot.readListBegin()
-          for _i861 in xrange(_size857):
-            _elem862 = iprot.readString()
-            self.names.append(_elem862)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = iprot.readString()
+            self.names.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22186,8 +22186,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 iter863 in self.names:
-        oprot.writeString(iter863)
+      for iter872 in self.names:
+        oprot.writeString(iter872)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22246,11 +22246,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype867, _size864) = iprot.readListBegin()
-          for _i868 in xrange(_size864):
-            _elem869 = Partition()
-            _elem869.read(iprot)
-            self.success.append(_elem869)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = Partition()
+            _elem878.read(iprot)
+            self.success.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22279,8 +22279,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 iter870 in self.success:
-        iter870.write(oprot)
+      for iter879 in self.success:
+        iter879.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22530,11 +22530,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype874, _size871) = iprot.readListBegin()
-          for _i875 in xrange(_size871):
-            _elem876 = Partition()
-            _elem876.read(iprot)
-            self.new_parts.append(_elem876)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = Partition()
+            _elem885.read(iprot)
+            self.new_parts.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22559,8 +22559,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 iter877 in self.new_parts:
-        iter877.write(oprot)
+      for iter886 in self.new_parts:
+        iter886.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22713,11 +22713,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype881, _size878) = iprot.readListBegin()
-          for _i882 in xrange(_size878):
-            _elem883 = Partition()
-            _elem883.read(iprot)
-            self.new_parts.append(_elem883)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = Partition()
+            _elem892.read(iprot)
+            self.new_parts.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22748,8 +22748,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 iter884 in self.new_parts:
-        iter884.write(oprot)
+      for iter893 in self.new_parts:
+        iter893.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23093,10 +23093,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype888, _size885) = iprot.readListBegin()
-          for _i889 in xrange(_size885):
-            _elem890 = iprot.readString()
-            self.part_vals.append(_elem890)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = iprot.readString()
+            self.part_vals.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23127,8 +23127,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 iter891 in self.part_vals:
-        oprot.writeString(iter891)
+      for iter900 in self.part_vals:
+        oprot.writeString(iter900)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -23270,10 +23270,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype895, _size892) = iprot.readListBegin()
-          for _i896 in xrange(_size892):
-            _elem897 = iprot.readString()
-            self.part_vals.append(_elem897)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.part_vals.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23295,8 +23295,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 iter898 in self.part_vals:
-        oprot.writeString(iter898)
+      for iter907 in self.part_vals:
+        oprot.writeString(iter907)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -23654,10 +23654,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype902, _size899) = iprot.readListBegin()
-          for _i903 in xrange(_size899):
-            _elem904 = iprot.readString()
-            self.success.append(_elem904)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = iprot.readString()
+            self.success.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23680,8 +23680,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 iter905 in self.success:
-        oprot.writeString(iter905)
+      for iter914 in self.success:
+        oprot.writeString(iter914)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23805,11 +23805,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype907, _vtype908, _size906 ) = iprot.readMapBegin()
-          for _i910 in xrange(_size906):
-            _key911 = iprot.readString()
-            _val912 = iprot.readString()
-            self.success[_key911] = _val912
+          (_ktype916, _vtype917, _size915 ) = iprot.readMapBegin()
+          for _i919 in xrange(_size915):
+            _key920 = iprot.readString()
+            _val921 = iprot.readString()
+            self.success[_key920] = _val921
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23832,9 +23832,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 kiter913,viter914 in self.success.items():
-        oprot.writeString(kiter913)
-        oprot.writeString(viter914)
+      for kiter922,viter923 in self.success.items():
+        oprot.writeString(kiter922)
+        oprot.writeString(viter923)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23910,11 +23910,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype916, _vtype917, _size915 ) = iprot.readMapBegin()
-          for _i919 in xrange(_size915):
-            _key920 = iprot.readString()
-            _val921 = iprot.readString()
-            self.part_vals[_key920] = _val921
+          (_ktype925, _vtype926, _size924 ) = iprot.readMapBegin()
+          for _i928 in xrange(_size924):
+            _key929 = iprot.readString()
+            _val930 = iprot.readString()
+            self.part_vals[_key929] = _val930
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23944,9 +23944,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 kiter922,viter923 in self.part_vals.items():
-        oprot.writeString(kiter922)
-        oprot.writeString(viter923)
+      for kiter931,viter932 in self.part_vals.items():
+        oprot.writeString(kiter931)
+        oprot.writeString(viter932)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -24160,11 +24160,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype925, _vtype926, _size924 ) = iprot.readMapBegin()
-          for _i928 in xrange(_size924):
-            _key929 = iprot.readString()
-            _val930 = iprot.readString()
-            self.part_vals[_key929] = _val930
+          (_ktype934, _vtype935, _size933 ) = iprot.readMapBegin()
+          for _i937 in xrange(_size933):
+            _key938 = iprot.readString()
+            _val939 = iprot.readString()
+            self.part_vals[_key938] = _val939
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24194,9 +24194,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 kiter931,viter932 in self.part_vals.items():
-        oprot.writeString(kiter931)
-        oprot.writeString(viter932)
+      for kiter940,viter941 in self.part_vals.items():
+        oprot.writeString(kiter940)
+        oprot.writeString(viter941)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25251,11 +25251,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype936, _size933) = iprot.readListBegin()
-          for _i937 in xrange(_size933):
-            _elem938 = Index()
-            _elem938.read(iprot)
-            self.success.append(_elem938)
+          (_etype945, _size942) = iprot.readListBegin()
+          for _i946 in xrange(_size942):
+            _elem947 = Index()
+            _elem947.read(iprot)
+            self.success.append(_elem947)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25284,8 +25284,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter939 in self.success:
-        iter939.write(oprot)
+      for iter948 in self.success:
+        iter948.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25440,10 +25440,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype943, _size940) = iprot.readListBegin()
-          for _i944 in xrange(_size940):
-            _elem945 = iprot.readString()
-            self.success.append(_elem945)
+          (_etype952, _size949) = iprot.readListBegin()
+          for _i953 in xrange(_size949):
+            _elem954 = iprot.readString()
+            self.success.append(_elem954)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25466,8 +25466,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter946 in self.success:
-        oprot.writeString(iter946)
+      for iter955 in self.success:
+        oprot.writeString(iter955)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -28333,10 +28333,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype950, _size947) = iprot.readListBegin()
-          for _i951 in xrange(_size947):
-            _elem952 = iprot.readString()
-            self.success.append(_elem952)
+          (_etype959, _size956) = iprot.readListBegin()
+          for _i960 in xrange(_size956):
+            _elem961 = iprot.readString()
+            self.success.append(_elem961)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28359,8 +28359,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 iter953 in self.success:
-        oprot.writeString(iter953)
+      for iter962 in self.success:
+        oprot.writeString(iter962)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29048,10 +29048,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype957, _size954) = iprot.readListBegin()
-          for _i958 in xrange(_size954):
-            _elem959 = iprot.readString()
-            self.success.append(_elem959)
+          (_etype966, _size963) = iprot.readListBegin()
+          for _i967 in xrange(_size963):
+            _elem968 = iprot.readString()
+            self.success.append(_elem968)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29074,8 +29074,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 iter960 in self.success:
-        oprot.writeString(iter960)
+      for iter969 in self.success:
+        oprot.writeString(iter969)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29589,11 +29589,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype964, _size961) = iprot.readListBegin()
-          for _i965 in xrange(_size961):
-            _elem966 = Role()
-            _elem966.read(iprot)
-            self.success.append(_elem966)
+          (_etype973, _size970) = iprot.readListBegin()
+          for _i974 in xrange(_size970):
+            _elem975 = Role()
+            _elem975.read(iprot)
+            self.success.append(_elem975)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29616,8 +29616,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 iter967 in self.success:
-        iter967.write(oprot)
+      for iter976 in self.success:
+        iter976.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30126,10 +30126,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype971, _size968) = iprot.readListBegin()
-          for _i972 in xrange(_size968):
-            _elem973 = iprot.readString()
-            self.group_names.append(_elem973)
+          (_etype980, _size977) = iprot.readListBegin()
+          for _i981 in xrange(_size977):
+            _elem982 = iprot.readString()
+            self.group_names.append(_elem982)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30154,8 +30154,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 iter974 in self.group_names:
-        oprot.writeString(iter974)
+      for iter983 in self.group_names:
+        oprot.writeString(iter983)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30382,11 +30382,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype978, _size975) = iprot.readListBegin()
-          for _i979 in xrange(_size975):
-            _elem980 = HiveObjectPrivilege()
-            _elem980.read(iprot)
-            self.success.append(_elem980)
+          (_etype987, _size984) = iprot.readListBegin()
+          for _i988 in xrange(_size984):
+            _elem989 = HiveObjectPrivilege()
+            _elem989.read(iprot)
+            self.success.append(_elem989)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30409,8 +30409,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 iter981 in self.success:
-        iter981.write(oprot)
+      for iter990 in self.success:
+        iter990.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30908,10 +30908,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype985, _size982) = iprot.readListBegin()
-          for _i986 in xrange(_size982):
-            _elem987 = iprot.readString()
-            self.group_names.append(_elem987)
+          (_etype994, _size991) = iprot.readListBegin()
+          for _i995 in xrange(_size991):
+            _elem996 = iprot.readString()
+            self.group_names.append(_elem996)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30932,8 +30932,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 iter988 in self.group_names:
-        oprot.writeString(iter988)
+      for iter997 in self.group_names:
+        oprot.writeString(iter997)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30988,10 +30988,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype992, _size989) = iprot.readListBegin()
-          for _i993 in xrange(_size989):
-            _elem994 = iprot.readString()
-            self.success.append(_elem994)
+          (_etype1001, _size998) = iprot.readListBegin()
+          for _i1002 in xrange(_size998):
+            _elem1003 = iprot.readString()
+            self.success.append(_elem1003)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31014,8 +31014,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 iter995 in self.success:
-        oprot.writeString(iter995)
+      for iter1004 in self.success:
+        oprot.writeString(iter1004)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31947,10 +31947,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype999, _size996) = iprot.readListBegin()
-          for _i1000 in xrange(_size996):
-            _elem1001 = iprot.readString()
-            self.success.append(_elem1001)
+          (_etype1008, _size1005) = iprot.readListBegin()
+          for _i1009 in xrange(_size1005):
+            _elem1010 = iprot.readString()
+            self.success.append(_elem1010)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31967,8 +31967,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 iter1002 in self.success:
-        oprot.writeString(iter1002)
+      for iter1011 in self.success:
+        oprot.writeString(iter1011)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -32495,10 +32495,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1006, _size1003) = iprot.readListBegin()
-          for _i1007 in xrange(_size1003):
-            _elem1008 = iprot.readString()
-            self.success.append(_elem1008)
+          (_etype1015, _size1012) = iprot.readListBegin()
+          for _i1016 in xrange(_size1012):
+            _elem1017 = iprot.readString()
+            self.success.append(_elem1017)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32515,8 +32515,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 iter1009 in self.success:
-        oprot.writeString(iter1009)
+      for iter1018 in self.success:
+        oprot.writeString(iter1018)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()


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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 763fe5e..6772338 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -713,13 +713,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list556 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list556.size);
-                String _elem557;
-                for (int _i558 = 0; _i558 < _list556.size; ++_i558)
+                org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list566.size);
+                String _elem567;
+                for (int _i568 = 0; _i568 < _list566.size; ++_i568)
                 {
-                  _elem557 = iprot.readString();
-                  struct.partitionVals.add(_elem557);
+                  _elem567 = iprot.readString();
+                  struct.partitionVals.add(_elem567);
                 }
                 iprot.readListEnd();
               }
@@ -768,9 +768,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter559 : struct.partitionVals)
+            for (String _iter569 : struct.partitionVals)
             {
-              oprot.writeString(_iter559);
+              oprot.writeString(_iter569);
             }
             oprot.writeListEnd();
           }
@@ -816,9 +816,9 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter560 : struct.partitionVals)
+          for (String _iter570 : struct.partitionVals)
           {
-            oprot.writeString(_iter560);
+            oprot.writeString(_iter570);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list561 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list561.size);
-          String _elem562;
-          for (int _i563 = 0; _i563 < _list561.size; ++_i563)
+          org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list571.size);
+          String _elem572;
+          for (int _i573 = 0; _i573 < _list571.size; ++_i573)
           {
-            _elem562 = iprot.readString();
-            struct.partitionVals.add(_elem562);
+            _elem572 = iprot.readString();
+            struct.partitionVals.add(_elem572);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index c30e7b8..f427a3a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list624 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list624.size);
-                Function _elem625;
-                for (int _i626 = 0; _i626 < _list624.size; ++_i626)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list634.size);
+                Function _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem625 = new Function();
-                  _elem625.read(iprot);
-                  struct.functions.add(_elem625);
+                  _elem635 = new Function();
+                  _elem635.read(iprot);
+                  struct.functions.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter627 : struct.functions)
+            for (Function _iter637 : struct.functions)
             {
-              _iter627.write(oprot);
+              _iter637.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter628 : struct.functions)
+          for (Function _iter638 : struct.functions)
           {
-            _iter628.write(oprot);
+            _iter638.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ public class GetAllFunctionsResponse implements org.apache.thrift.TBase<GetAllFu
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list629 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list629.size);
-          Function _elem630;
-          for (int _i631 = 0; _i631 < _list629.size; ++_i631)
+          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list639.size);
+          Function _elem640;
+          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
           {
-            _elem630 = new Function();
-            _elem630.read(iprot);
-            struct.functions.add(_elem630);
+            _elem640 = new Function();
+            _elem640.read(iprot);
+            struct.functions.add(_elem640);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 4ba95ba..1ea90a0 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list574.size);
-                long _elem575;
-                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                org.apache.thrift.protocol.TList _list584 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list584.size);
+                long _elem585;
+                for (int _i586 = 0; _i586 < _list584.size; ++_i586)
                 {
-                  _elem575 = iprot.readI64();
-                  struct.fileIds.add(_elem575);
+                  _elem585 = iprot.readI64();
+                  struct.fileIds.add(_elem585);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
         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 _iter577 : struct.fileIds)
+          for (long _iter587 : struct.fileIds)
           {
-            oprot.writeI64(_iter577);
+            oprot.writeI64(_iter587);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter578 : struct.fileIds)
+        for (long _iter588 : struct.fileIds)
         {
-          oprot.writeI64(_iter578);
+          oprot.writeI64(_iter588);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ public class GetFileMetadataByExprRequest implements org.apache.thrift.TBase<Get
     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 _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list579.size);
-        long _elem580;
-        for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+        org.apache.thrift.protocol.TList _list589 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list589.size);
+        long _elem590;
+        for (int _i591 = 0; _i591 < _list589.size; ++_i591)
         {
-          _elem580 = iprot.readI64();
-          struct.fileIds.add(_elem580);
+          _elem590 = iprot.readI64();
+          struct.fileIds.add(_elem590);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index 2e0bd9c..609abc5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map564 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map564.size);
-                long _key565;
-                MetadataPpdResult _val566;
-                for (int _i567 = 0; _i567 < _map564.size; ++_i567)
+                org.apache.thrift.protocol.TMap _map574 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map574.size);
+                long _key575;
+                MetadataPpdResult _val576;
+                for (int _i577 = 0; _i577 < _map574.size; ++_i577)
                 {
-                  _key565 = iprot.readI64();
-                  _val566 = new MetadataPpdResult();
-                  _val566.read(iprot);
-                  struct.metadata.put(_key565, _val566);
+                  _key575 = iprot.readI64();
+                  _val576 = new MetadataPpdResult();
+                  _val576.read(iprot);
+                  struct.metadata.put(_key575, _val576);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
         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> _iter568 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter578 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter568.getKey());
-            _iter568.getValue().write(oprot);
+            oprot.writeI64(_iter578.getKey());
+            _iter578.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter569 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter579 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter569.getKey());
-          _iter569.getValue().write(oprot);
+          oprot.writeI64(_iter579.getKey());
+          _iter579.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ public class GetFileMetadataByExprResult implements org.apache.thrift.TBase<GetF
     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 _map570 = 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*_map570.size);
-        long _key571;
-        MetadataPpdResult _val572;
-        for (int _i573 = 0; _i573 < _map570.size; ++_i573)
+        org.apache.thrift.protocol.TMap _map580 = 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*_map580.size);
+        long _key581;
+        MetadataPpdResult _val582;
+        for (int _i583 = 0; _i583 < _map580.size; ++_i583)
         {
-          _key571 = iprot.readI64();
-          _val572 = new MetadataPpdResult();
-          _val572.read(iprot);
-          struct.metadata.put(_key571, _val572);
+          _key581 = iprot.readI64();
+          _val582 = new MetadataPpdResult();
+          _val582.read(iprot);
+          struct.metadata.put(_key581, _val582);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index c079ecd..5cf880a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list592 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list592.size);
-                long _elem593;
-                for (int _i594 = 0; _i594 < _list592.size; ++_i594)
+                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list602.size);
+                long _elem603;
+                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
                 {
-                  _elem593 = iprot.readI64();
-                  struct.fileIds.add(_elem593);
+                  _elem603 = iprot.readI64();
+                  struct.fileIds.add(_elem603);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
         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 _iter595 : struct.fileIds)
+          for (long _iter605 : struct.fileIds)
           {
-            oprot.writeI64(_iter595);
+            oprot.writeI64(_iter605);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter596 : struct.fileIds)
+        for (long _iter606 : struct.fileIds)
         {
-          oprot.writeI64(_iter596);
+          oprot.writeI64(_iter606);
         }
       }
     }
@@ -421,13 +421,13 @@ public class GetFileMetadataRequest implements org.apache.thrift.TBase<GetFileMe
     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 _list597 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list597.size);
-        long _elem598;
-        for (int _i599 = 0; _i599 < _list597.size; ++_i599)
+        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list607.size);
+        long _elem608;
+        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
         {
-          _elem598 = iprot.readI64();
-          struct.fileIds.add(_elem598);
+          _elem608 = iprot.readI64();
+          struct.fileIds.add(_elem608);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index fb2a64a..8870c2f 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map582 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map582.size);
-                long _key583;
-                ByteBuffer _val584;
-                for (int _i585 = 0; _i585 < _map582.size; ++_i585)
+                org.apache.thrift.protocol.TMap _map592 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map592.size);
+                long _key593;
+                ByteBuffer _val594;
+                for (int _i595 = 0; _i595 < _map592.size; ++_i595)
                 {
-                  _key583 = iprot.readI64();
-                  _val584 = iprot.readBinary();
-                  struct.metadata.put(_key583, _val584);
+                  _key593 = iprot.readI64();
+                  _val594 = iprot.readBinary();
+                  struct.metadata.put(_key593, _val594);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
         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> _iter586 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter596 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter586.getKey());
-            oprot.writeBinary(_iter586.getValue());
+            oprot.writeI64(_iter596.getKey());
+            oprot.writeBinary(_iter596.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter587 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter597 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter587.getKey());
-          oprot.writeBinary(_iter587.getValue());
+          oprot.writeI64(_iter597.getKey());
+          oprot.writeBinary(_iter597.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ public class GetFileMetadataResult implements org.apache.thrift.TBase<GetFileMet
     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 _map588 = 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*_map588.size);
-        long _key589;
-        ByteBuffer _val590;
-        for (int _i591 = 0; _i591 < _map588.size; ++_i591)
+        org.apache.thrift.protocol.TMap _map598 = 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*_map598.size);
+        long _key599;
+        ByteBuffer _val600;
+        for (int _i601 = 0; _i601 < _map598.size; ++_i601)
         {
-          _key589 = iprot.readI64();
-          _val590 = iprot.readBinary();
-          struct.metadata.put(_key589, _val590);
+          _key599 = iprot.readI64();
+          _val600 = iprot.readBinary();
+          struct.metadata.put(_key599, _val600);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 820a573..a8df524 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -351,13 +351,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 1: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list548.size);
-                String _elem549;
-                for (int _i550 = 0; _i550 < _list548.size; ++_i550)
+                org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list558.size);
+                String _elem559;
+                for (int _i560 = 0; _i560 < _list558.size; ++_i560)
                 {
-                  _elem549 = iprot.readString();
-                  struct.filesAdded.add(_elem549);
+                  _elem559 = iprot.readString();
+                  struct.filesAdded.add(_elem559);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter551 : struct.filesAdded)
+          for (String _iter561 : struct.filesAdded)
           {
-            oprot.writeString(_iter551);
+            oprot.writeString(_iter561);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter552 : struct.filesAdded)
+        for (String _iter562 : struct.filesAdded)
         {
-          oprot.writeString(_iter552);
+          oprot.writeString(_iter562);
         }
       }
     }
@@ -421,13 +421,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list553.size);
-        String _elem554;
-        for (int _i555 = 0; _i555 < _list553.size; ++_i555)
+        org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list563.size);
+        String _elem564;
+        for (int _i565 = 0; _i565 < _list563.size; ++_i565)
         {
-          _elem554 = iprot.readString();
-          struct.filesAdded.add(_elem554);
+          _elem564 = iprot.readString();
+          struct.filesAdded.add(_elem564);
         }
       }
       struct.setFilesAddedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index ce8aca3..edc548a 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list540.size);
-                NotificationEvent _elem541;
-                for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                org.apache.thrift.protocol.TList _list550 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list550.size);
+                NotificationEvent _elem551;
+                for (int _i552 = 0; _i552 < _list550.size; ++_i552)
                 {
-                  _elem541 = new NotificationEvent();
-                  _elem541.read(iprot);
-                  struct.events.add(_elem541);
+                  _elem551 = new NotificationEvent();
+                  _elem551.read(iprot);
+                  struct.events.add(_elem551);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter543 : struct.events)
+          for (NotificationEvent _iter553 : struct.events)
           {
-            _iter543.write(oprot);
+            _iter553.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter544 : struct.events)
+        for (NotificationEvent _iter554 : struct.events)
         {
-          _iter544.write(oprot);
+          _iter554.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class NotificationEventResponse implements org.apache.thrift.TBase<Notifi
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list545.size);
-        NotificationEvent _elem546;
-        for (int _i547 = 0; _i547 < _list545.size; ++_i547)
+        org.apache.thrift.protocol.TList _list555 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list555.size);
+        NotificationEvent _elem556;
+        for (int _i557 = 0; _i557 < _list555.size; ++_i557)
         {
-          _elem546 = new NotificationEvent();
-          _elem546.read(iprot);
-          struct.events.add(_elem546);
+          _elem556 = new NotificationEvent();
+          _elem556.read(iprot);
+          struct.events.add(_elem556);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 9323d9f..6eff25d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list600 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list600.size);
-                long _elem601;
-                for (int _i602 = 0; _i602 < _list600.size; ++_i602)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list610.size);
+                long _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem601 = iprot.readI64();
-                  struct.fileIds.add(_elem601);
+                  _elem611 = iprot.readI64();
+                  struct.fileIds.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list603 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list603.size);
-                ByteBuffer _elem604;
-                for (int _i605 = 0; _i605 < _list603.size; ++_i605)
+                org.apache.thrift.protocol.TList _list613 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list613.size);
+                ByteBuffer _elem614;
+                for (int _i615 = 0; _i615 < _list613.size; ++_i615)
                 {
-                  _elem604 = iprot.readBinary();
-                  struct.metadata.add(_elem604);
+                  _elem614 = iprot.readBinary();
+                  struct.metadata.add(_elem614);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         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 _iter606 : struct.fileIds)
+          for (long _iter616 : struct.fileIds)
           {
-            oprot.writeI64(_iter606);
+            oprot.writeI64(_iter616);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter607 : struct.metadata)
+          for (ByteBuffer _iter617 : struct.metadata)
           {
-            oprot.writeBinary(_iter607);
+            oprot.writeBinary(_iter617);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter608 : struct.fileIds)
+        for (long _iter618 : struct.fileIds)
         {
-          oprot.writeI64(_iter608);
+          oprot.writeI64(_iter618);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter609 : struct.metadata)
+        for (ByteBuffer _iter619 : struct.metadata)
         {
-          oprot.writeBinary(_iter609);
+          oprot.writeBinary(_iter619);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     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 _list610 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list610.size);
-        long _elem611;
-        for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+        org.apache.thrift.protocol.TList _list620 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list620.size);
+        long _elem621;
+        for (int _i622 = 0; _i622 < _list620.size; ++_i622)
         {
-          _elem611 = iprot.readI64();
-          struct.fileIds.add(_elem611);
+          _elem621 = iprot.readI64();
+          struct.fileIds.add(_elem621);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list613 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list613.size);
-        ByteBuffer _elem614;
-        for (int _i615 = 0; _i615 < _list613.size; ++_i615)
+        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list623.size);
+        ByteBuffer _elem624;
+        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
         {
-          _elem614 = iprot.readBinary();
-          struct.metadata.add(_elem614);
+          _elem624 = iprot.readBinary();
+          struct.metadata.add(_elem624);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index 6641c7e..ed86165 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list524.size);
-                ShowCompactResponseElement _elem525;
-                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                org.apache.thrift.protocol.TList _list534 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list534.size);
+                ShowCompactResponseElement _elem535;
+                for (int _i536 = 0; _i536 < _list534.size; ++_i536)
                 {
-                  _elem525 = new ShowCompactResponseElement();
-                  _elem525.read(iprot);
-                  struct.compacts.add(_elem525);
+                  _elem535 = new ShowCompactResponseElement();
+                  _elem535.read(iprot);
+                  struct.compacts.add(_elem535);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter527 : struct.compacts)
+          for (ShowCompactResponseElement _iter537 : struct.compacts)
           {
-            _iter527.write(oprot);
+            _iter537.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter528 : struct.compacts)
+        for (ShowCompactResponseElement _iter538 : struct.compacts)
         {
-          _iter528.write(oprot);
+          _iter538.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class ShowCompactResponse implements org.apache.thrift.TBase<ShowCompactR
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list529.size);
-        ShowCompactResponseElement _elem530;
-        for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+        org.apache.thrift.protocol.TList _list539 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list539.size);
+        ShowCompactResponseElement _elem540;
+        for (int _i541 = 0; _i541 < _list539.size; ++_i541)
         {
-          _elem530 = new ShowCompactResponseElement();
-          _elem530.read(iprot);
-          struct.compacts.add(_elem530);
+          _elem540 = new ShowCompactResponseElement();
+          _elem540.read(iprot);
+          struct.compacts.add(_elem540);
         }
       }
       struct.setCompactsIsSet(true);


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

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c57a5961/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 3c9e038..2d82c92 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -10792,14 +10792,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size560 = 0;
-            $_etype563 = 0;
-            $xfer += $input->readListBegin($_etype563, $_size560);
-            for ($_i564 = 0; $_i564 < $_size560; ++$_i564)
+            $_size569 = 0;
+            $_etype572 = 0;
+            $xfer += $input->readListBegin($_etype572, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $elem565 = null;
-              $xfer += $input->readString($elem565);
-              $this->success []= $elem565;
+              $elem574 = null;
+              $xfer += $input->readString($elem574);
+              $this->success []= $elem574;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10835,9 +10835,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter566)
+          foreach ($this->success as $iter575)
           {
-            $xfer += $output->writeString($iter566);
+            $xfer += $output->writeString($iter575);
           }
         }
         $output->writeListEnd();
@@ -10968,14 +10968,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size567 = 0;
-            $_etype570 = 0;
-            $xfer += $input->readListBegin($_etype570, $_size567);
-            for ($_i571 = 0; $_i571 < $_size567; ++$_i571)
+            $_size576 = 0;
+            $_etype579 = 0;
+            $xfer += $input->readListBegin($_etype579, $_size576);
+            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
             {
-              $elem572 = null;
-              $xfer += $input->readString($elem572);
-              $this->success []= $elem572;
+              $elem581 = null;
+              $xfer += $input->readString($elem581);
+              $this->success []= $elem581;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11011,9 +11011,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter573)
+          foreach ($this->success as $iter582)
           {
-            $xfer += $output->writeString($iter573);
+            $xfer += $output->writeString($iter582);
           }
         }
         $output->writeListEnd();
@@ -12014,18 +12014,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size574 = 0;
-            $_ktype575 = 0;
-            $_vtype576 = 0;
-            $xfer += $input->readMapBegin($_ktype575, $_vtype576, $_size574);
-            for ($_i578 = 0; $_i578 < $_size574; ++$_i578)
+            $_size583 = 0;
+            $_ktype584 = 0;
+            $_vtype585 = 0;
+            $xfer += $input->readMapBegin($_ktype584, $_vtype585, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
             {
-              $key579 = '';
-              $val580 = new \metastore\Type();
-              $xfer += $input->readString($key579);
-              $val580 = new \metastore\Type();
-              $xfer += $val580->read($input);
-              $this->success[$key579] = $val580;
+              $key588 = '';
+              $val589 = new \metastore\Type();
+              $xfer += $input->readString($key588);
+              $val589 = new \metastore\Type();
+              $xfer += $val589->read($input);
+              $this->success[$key588] = $val589;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -12061,10 +12061,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter581 => $viter582)
+          foreach ($this->success as $kiter590 => $viter591)
           {
-            $xfer += $output->writeString($kiter581);
-            $xfer += $viter582->write($output);
+            $xfer += $output->writeString($kiter590);
+            $xfer += $viter591->write($output);
           }
         }
         $output->writeMapEnd();
@@ -12268,15 +12268,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size583 = 0;
-            $_etype586 = 0;
-            $xfer += $input->readListBegin($_etype586, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size592 = 0;
+            $_etype595 = 0;
+            $xfer += $input->readListBegin($_etype595, $_size592);
+            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
             {
-              $elem588 = null;
-              $elem588 = new \metastore\FieldSchema();
-              $xfer += $elem588->read($input);
-              $this->success []= $elem588;
+              $elem597 = null;
+              $elem597 = new \metastore\FieldSchema();
+              $xfer += $elem597->read($input);
+              $this->success []= $elem597;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12328,9 +12328,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter589)
+          foreach ($this->success as $iter598)
           {
-            $xfer += $iter589->write($output);
+            $xfer += $iter598->write($output);
           }
         }
         $output->writeListEnd();
@@ -12572,15 +12572,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size590 = 0;
-            $_etype593 = 0;
-            $xfer += $input->readListBegin($_etype593, $_size590);
-            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
+            $_size599 = 0;
+            $_etype602 = 0;
+            $xfer += $input->readListBegin($_etype602, $_size599);
+            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
             {
-              $elem595 = null;
-              $elem595 = new \metastore\FieldSchema();
-              $xfer += $elem595->read($input);
-              $this->success []= $elem595;
+              $elem604 = null;
+              $elem604 = new \metastore\FieldSchema();
+              $xfer += $elem604->read($input);
+              $this->success []= $elem604;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12632,9 +12632,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter596)
+          foreach ($this->success as $iter605)
           {
-            $xfer += $iter596->write($output);
+            $xfer += $iter605->write($output);
           }
         }
         $output->writeListEnd();
@@ -12848,15 +12848,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size597 = 0;
-            $_etype600 = 0;
-            $xfer += $input->readListBegin($_etype600, $_size597);
-            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
+            $_size606 = 0;
+            $_etype609 = 0;
+            $xfer += $input->readListBegin($_etype609, $_size606);
+            for ($_i610 = 0; $_i610 < $_size606; ++$_i610)
             {
-              $elem602 = null;
-              $elem602 = new \metastore\FieldSchema();
-              $xfer += $elem602->read($input);
-              $this->success []= $elem602;
+              $elem611 = null;
+              $elem611 = new \metastore\FieldSchema();
+              $xfer += $elem611->read($input);
+              $this->success []= $elem611;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12908,9 +12908,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter603)
+          foreach ($this->success as $iter612)
           {
-            $xfer += $iter603->write($output);
+            $xfer += $iter612->write($output);
           }
         }
         $output->writeListEnd();
@@ -13152,15 +13152,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size604 = 0;
-            $_etype607 = 0;
-            $xfer += $input->readListBegin($_etype607, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size613 = 0;
+            $_etype616 = 0;
+            $xfer += $input->readListBegin($_etype616, $_size613);
+            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
             {
-              $elem609 = null;
-              $elem609 = new \metastore\FieldSchema();
-              $xfer += $elem609->read($input);
-              $this->success []= $elem609;
+              $elem618 = null;
+              $elem618 = new \metastore\FieldSchema();
+              $xfer += $elem618->read($input);
+              $this->success []= $elem618;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13212,9 +13212,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter610)
+          foreach ($this->success as $iter619)
           {
-            $xfer += $iter610->write($output);
+            $xfer += $iter619->write($output);
           }
         }
         $output->writeListEnd();
@@ -13822,15 +13822,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size620 = 0;
+            $_etype623 = 0;
+            $xfer += $input->readListBegin($_etype623, $_size620);
+            for ($_i624 = 0; $_i624 < $_size620; ++$_i624)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem616->read($input);
-              $this->primaryKeys []= $elem616;
+              $elem625 = null;
+              $elem625 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem625->read($input);
+              $this->primaryKeys []= $elem625;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13840,15 +13840,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size617 = 0;
-            $_etype620 = 0;
-            $xfer += $input->readListBegin($_etype620, $_size617);
-            for ($_i621 = 0; $_i621 < $_size617; ++$_i621)
+            $_size626 = 0;
+            $_etype629 = 0;
+            $xfer += $input->readListBegin($_etype629, $_size626);
+            for ($_i630 = 0; $_i630 < $_size626; ++$_i630)
             {
-              $elem622 = null;
-              $elem622 = new \metastore\SQLForeignKey();
-              $xfer += $elem622->read($input);
-              $this->foreignKeys []= $elem622;
+              $elem631 = null;
+              $elem631 = new \metastore\SQLForeignKey();
+              $xfer += $elem631->read($input);
+              $this->foreignKeys []= $elem631;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13884,9 +13884,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter623)
+          foreach ($this->primaryKeys as $iter632)
           {
-            $xfer += $iter623->write($output);
+            $xfer += $iter632->write($output);
           }
         }
         $output->writeListEnd();
@@ -13901,9 +13901,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter624)
+          foreach ($this->foreignKeys as $iter633)
           {
-            $xfer += $iter624->write($output);
+            $xfer += $iter633->write($output);
           }
         }
         $output->writeListEnd();
@@ -15249,14 +15249,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readListBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size634 = 0;
+            $_etype637 = 0;
+            $xfer += $input->readListBegin($_etype637, $_size634);
+            for ($_i638 = 0; $_i638 < $_size634; ++$_i638)
             {
-              $elem630 = null;
-              $xfer += $input->readString($elem630);
-              $this->success []= $elem630;
+              $elem639 = null;
+              $xfer += $input->readString($elem639);
+              $this->success []= $elem639;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15292,9 +15292,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter631)
+          foreach ($this->success as $iter640)
           {
-            $xfer += $output->writeString($iter631);
+            $xfer += $output->writeString($iter640);
           }
         }
         $output->writeListEnd();
@@ -15399,14 +15399,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
             {
-              $elem637 = null;
-              $xfer += $input->readString($elem637);
-              $this->tbl_types []= $elem637;
+              $elem646 = null;
+              $xfer += $input->readString($elem646);
+              $this->tbl_types []= $elem646;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15444,9 +15444,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter638)
+          foreach ($this->tbl_types as $iter647)
           {
-            $xfer += $output->writeString($iter638);
+            $xfer += $output->writeString($iter647);
           }
         }
         $output->writeListEnd();
@@ -15523,15 +15523,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size639 = 0;
-            $_etype642 = 0;
-            $xfer += $input->readListBegin($_etype642, $_size639);
-            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem644 = null;
-              $elem644 = new \metastore\TableMeta();
-              $xfer += $elem644->read($input);
-              $this->success []= $elem644;
+              $elem653 = null;
+              $elem653 = new \metastore\TableMeta();
+              $xfer += $elem653->read($input);
+              $this->success []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15567,9 +15567,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter645)
+          foreach ($this->success as $iter654)
           {
-            $xfer += $iter645->write($output);
+            $xfer += $iter654->write($output);
           }
         }
         $output->writeListEnd();
@@ -15725,14 +15725,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size655 = 0;
+            $_etype658 = 0;
+            $xfer += $input->readListBegin($_etype658, $_size655);
+            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->success []= $elem651;
+              $elem660 = null;
+              $xfer += $input->readString($elem660);
+              $this->success []= $elem660;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15768,9 +15768,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter652)
+          foreach ($this->success as $iter661)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter661);
           }
         }
         $output->writeListEnd();
@@ -16085,14 +16085,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem658 = null;
-              $xfer += $input->readString($elem658);
-              $this->tbl_names []= $elem658;
+              $elem667 = null;
+              $xfer += $input->readString($elem667);
+              $this->tbl_names []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16125,9 +16125,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter659)
+          foreach ($this->tbl_names as $iter668)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter668);
           }
         }
         $output->writeListEnd();
@@ -16228,15 +16228,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size660 = 0;
-            $_etype663 = 0;
-            $xfer += $input->readListBegin($_etype663, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem665 = null;
-              $elem665 = new \metastore\Table();
-              $xfer += $elem665->read($input);
-              $this->success []= $elem665;
+              $elem674 = null;
+              $elem674 = new \metastore\Table();
+              $xfer += $elem674->read($input);
+              $this->success []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16288,9 +16288,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter666)
+          foreach ($this->success as $iter675)
           {
-            $xfer += $iter666->write($output);
+            $xfer += $iter675->write($output);
           }
         }
         $output->writeListEnd();
@@ -16526,14 +16526,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size667 = 0;
-            $_etype670 = 0;
-            $xfer += $input->readListBegin($_etype670, $_size667);
-            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem672 = null;
-              $xfer += $input->readString($elem672);
-              $this->success []= $elem672;
+              $elem681 = null;
+              $xfer += $input->readString($elem681);
+              $this->success []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16585,9 +16585,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter673)
+          foreach ($this->success as $iter682)
           {
-            $xfer += $output->writeString($iter673);
+            $xfer += $output->writeString($iter682);
           }
         }
         $output->writeListEnd();
@@ -17900,15 +17900,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size674 = 0;
-            $_etype677 = 0;
-            $xfer += $input->readListBegin($_etype677, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem679 = null;
-              $elem679 = new \metastore\Partition();
-              $xfer += $elem679->read($input);
-              $this->new_parts []= $elem679;
+              $elem688 = null;
+              $elem688 = new \metastore\Partition();
+              $xfer += $elem688->read($input);
+              $this->new_parts []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17936,9 +17936,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter680)
+          foreach ($this->new_parts as $iter689)
           {
-            $xfer += $iter680->write($output);
+            $xfer += $iter689->write($output);
           }
         }
         $output->writeListEnd();
@@ -18153,15 +18153,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size681 = 0;
-            $_etype684 = 0;
-            $xfer += $input->readListBegin($_etype684, $_size681);
-            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem686 = null;
-              $elem686 = new \metastore\PartitionSpec();
-              $xfer += $elem686->read($input);
-              $this->new_parts []= $elem686;
+              $elem695 = null;
+              $elem695 = new \metastore\PartitionSpec();
+              $xfer += $elem695->read($input);
+              $this->new_parts []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18189,9 +18189,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter687)
+          foreach ($this->new_parts as $iter696)
           {
-            $xfer += $iter687->write($output);
+            $xfer += $iter696->write($output);
           }
         }
         $output->writeListEnd();
@@ -18441,14 +18441,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size688 = 0;
-            $_etype691 = 0;
-            $xfer += $input->readListBegin($_etype691, $_size688);
-            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem693 = null;
-              $xfer += $input->readString($elem693);
-              $this->part_vals []= $elem693;
+              $elem702 = null;
+              $xfer += $input->readString($elem702);
+              $this->part_vals []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18486,9 +18486,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter694)
+          foreach ($this->part_vals as $iter703)
           {
-            $xfer += $output->writeString($iter694);
+            $xfer += $output->writeString($iter703);
           }
         }
         $output->writeListEnd();
@@ -18990,14 +18990,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size695 = 0;
-            $_etype698 = 0;
-            $xfer += $input->readListBegin($_etype698, $_size695);
-            for ($_i699 = 0; $_i699 < $_size695; ++$_i699)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem700 = null;
-              $xfer += $input->readString($elem700);
-              $this->part_vals []= $elem700;
+              $elem709 = null;
+              $xfer += $input->readString($elem709);
+              $this->part_vals []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19043,9 +19043,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter701)
+          foreach ($this->part_vals as $iter710)
           {
-            $xfer += $output->writeString($iter701);
+            $xfer += $output->writeString($iter710);
           }
         }
         $output->writeListEnd();
@@ -19899,14 +19899,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size702 = 0;
-            $_etype705 = 0;
-            $xfer += $input->readListBegin($_etype705, $_size702);
-            for ($_i706 = 0; $_i706 < $_size702; ++$_i706)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem707 = null;
-              $xfer += $input->readString($elem707);
-              $this->part_vals []= $elem707;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->part_vals []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19951,9 +19951,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter708)
+          foreach ($this->part_vals as $iter717)
           {
-            $xfer += $output->writeString($iter708);
+            $xfer += $output->writeString($iter717);
           }
         }
         $output->writeListEnd();
@@ -20206,14 +20206,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size709 = 0;
-            $_etype712 = 0;
-            $xfer += $input->readListBegin($_etype712, $_size709);
-            for ($_i713 = 0; $_i713 < $_size709; ++$_i713)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem714 = null;
-              $xfer += $input->readString($elem714);
-              $this->part_vals []= $elem714;
+              $elem723 = null;
+              $xfer += $input->readString($elem723);
+              $this->part_vals []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20266,9 +20266,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter715)
+          foreach ($this->part_vals as $iter724)
           {
-            $xfer += $output->writeString($iter715);
+            $xfer += $output->writeString($iter724);
           }
         }
         $output->writeListEnd();
@@ -21282,14 +21282,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size716 = 0;
-            $_etype719 = 0;
-            $xfer += $input->readListBegin($_etype719, $_size716);
-            for ($_i720 = 0; $_i720 < $_size716; ++$_i720)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem721 = null;
-              $xfer += $input->readString($elem721);
-              $this->part_vals []= $elem721;
+              $elem730 = null;
+              $xfer += $input->readString($elem730);
+              $this->part_vals []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21327,9 +21327,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter722)
+          foreach ($this->part_vals as $iter731)
           {
-            $xfer += $output->writeString($iter722);
+            $xfer += $output->writeString($iter731);
           }
         }
         $output->writeListEnd();
@@ -21571,17 +21571,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size723 = 0;
-            $_ktype724 = 0;
-            $_vtype725 = 0;
-            $xfer += $input->readMapBegin($_ktype724, $_vtype725, $_size723);
-            for ($_i727 = 0; $_i727 < $_size723; ++$_i727)
+            $_size732 = 0;
+            $_ktype733 = 0;
+            $_vtype734 = 0;
+            $xfer += $input->readMapBegin($_ktype733, $_vtype734, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $key728 = '';
-              $val729 = '';
-              $xfer += $input->readString($key728);
-              $xfer += $input->readString($val729);
-              $this->partitionSpecs[$key728] = $val729;
+              $key737 = '';
+              $val738 = '';
+              $xfer += $input->readString($key737);
+              $xfer += $input->readString($val738);
+              $this->partitionSpecs[$key737] = $val738;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -21637,10 +21637,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter730 => $viter731)
+          foreach ($this->partitionSpecs as $kiter739 => $viter740)
           {
-            $xfer += $output->writeString($kiter730);
-            $xfer += $output->writeString($viter731);
+            $xfer += $output->writeString($kiter739);
+            $xfer += $output->writeString($viter740);
           }
         }
         $output->writeMapEnd();
@@ -21952,17 +21952,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size732 = 0;
-            $_ktype733 = 0;
-            $_vtype734 = 0;
-            $xfer += $input->readMapBegin($_ktype733, $_vtype734, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size741 = 0;
+            $_ktype742 = 0;
+            $_vtype743 = 0;
+            $xfer += $input->readMapBegin($_ktype742, $_vtype743, $_size741);
+            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
             {
-              $key737 = '';
-              $val738 = '';
-              $xfer += $input->readString($key737);
-              $xfer += $input->readString($val738);
-              $this->partitionSpecs[$key737] = $val738;
+              $key746 = '';
+              $val747 = '';
+              $xfer += $input->readString($key746);
+              $xfer += $input->readString($val747);
+              $this->partitionSpecs[$key746] = $val747;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22018,10 +22018,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter739 => $viter740)
+          foreach ($this->partitionSpecs as $kiter748 => $viter749)
           {
-            $xfer += $output->writeString($kiter739);
-            $xfer += $output->writeString($viter740);
+            $xfer += $output->writeString($kiter748);
+            $xfer += $output->writeString($viter749);
           }
         }
         $output->writeMapEnd();
@@ -22154,15 +22154,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size741 = 0;
-            $_etype744 = 0;
-            $xfer += $input->readListBegin($_etype744, $_size741);
-            for ($_i745 = 0; $_i745 < $_size741; ++$_i745)
+            $_size750 = 0;
+            $_etype753 = 0;
+            $xfer += $input->readListBegin($_etype753, $_size750);
+            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
             {
-              $elem746 = null;
-              $elem746 = new \metastore\Partition();
-              $xfer += $elem746->read($input);
-              $this->success []= $elem746;
+              $elem755 = null;
+              $elem755 = new \metastore\Partition();
+              $xfer += $elem755->read($input);
+              $this->success []= $elem755;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22222,9 +22222,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter747)
+          foreach ($this->success as $iter756)
           {
-            $xfer += $iter747->write($output);
+            $xfer += $iter756->write($output);
           }
         }
         $output->writeListEnd();
@@ -22370,14 +22370,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size748 = 0;
-            $_etype751 = 0;
-            $xfer += $input->readListBegin($_etype751, $_size748);
-            for ($_i752 = 0; $_i752 < $_size748; ++$_i752)
+            $_size757 = 0;
+            $_etype760 = 0;
+            $xfer += $input->readListBegin($_etype760, $_size757);
+            for ($_i761 = 0; $_i761 < $_size757; ++$_i761)
             {
-              $elem753 = null;
-              $xfer += $input->readString($elem753);
-              $this->part_vals []= $elem753;
+              $elem762 = null;
+              $xfer += $input->readString($elem762);
+              $this->part_vals []= $elem762;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22394,14 +22394,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size754 = 0;
-            $_etype757 = 0;
-            $xfer += $input->readListBegin($_etype757, $_size754);
-            for ($_i758 = 0; $_i758 < $_size754; ++$_i758)
+            $_size763 = 0;
+            $_etype766 = 0;
+            $xfer += $input->readListBegin($_etype766, $_size763);
+            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
             {
-              $elem759 = null;
-              $xfer += $input->readString($elem759);
-              $this->group_names []= $elem759;
+              $elem768 = null;
+              $xfer += $input->readString($elem768);
+              $this->group_names []= $elem768;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22439,9 +22439,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter760)
+          foreach ($this->part_vals as $iter769)
           {
-            $xfer += $output->writeString($iter760);
+            $xfer += $output->writeString($iter769);
           }
         }
         $output->writeListEnd();
@@ -22461,9 +22461,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter761)
+          foreach ($this->group_names as $iter770)
           {
-            $xfer += $output->writeString($iter761);
+            $xfer += $output->writeString($iter770);
           }
         }
         $output->writeListEnd();
@@ -23054,15 +23054,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size771 = 0;
+            $_etype774 = 0;
+            $xfer += $input->readListBegin($_etype774, $_size771);
+            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\Partition();
-              $xfer += $elem767->read($input);
-              $this->success []= $elem767;
+              $elem776 = null;
+              $elem776 = new \metastore\Partition();
+              $xfer += $elem776->read($input);
+              $this->success []= $elem776;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23106,9 +23106,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter768)
+          foreach ($this->success as $iter777)
           {
-            $xfer += $iter768->write($output);
+            $xfer += $iter777->write($output);
           }
         }
         $output->writeListEnd();
@@ -23254,14 +23254,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size769 = 0;
-            $_etype772 = 0;
-            $xfer += $input->readListBegin($_etype772, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
             {
-              $elem774 = null;
-              $xfer += $input->readString($elem774);
-              $this->group_names []= $elem774;
+              $elem783 = null;
+              $xfer += $input->readString($elem783);
+              $this->group_names []= $elem783;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23309,9 +23309,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter775)
+          foreach ($this->group_names as $iter784)
           {
-            $xfer += $output->writeString($iter775);
+            $xfer += $output->writeString($iter784);
           }
         }
         $output->writeListEnd();
@@ -23400,15 +23400,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size776 = 0;
-            $_etype779 = 0;
-            $xfer += $input->readListBegin($_etype779, $_size776);
-            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
             {
-              $elem781 = null;
-              $elem781 = new \metastore\Partition();
-              $xfer += $elem781->read($input);
-              $this->success []= $elem781;
+              $elem790 = null;
+              $elem790 = new \metastore\Partition();
+              $xfer += $elem790->read($input);
+              $this->success []= $elem790;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23452,9 +23452,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter782)
+          foreach ($this->success as $iter791)
           {
-            $xfer += $iter782->write($output);
+            $xfer += $iter791->write($output);
           }
         }
         $output->writeListEnd();
@@ -23674,15 +23674,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size783 = 0;
-            $_etype786 = 0;
-            $xfer += $input->readListBegin($_etype786, $_size783);
-            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem788 = null;
-              $elem788 = new \metastore\PartitionSpec();
-              $xfer += $elem788->read($input);
-              $this->success []= $elem788;
+              $elem797 = null;
+              $elem797 = new \metastore\PartitionSpec();
+              $xfer += $elem797->read($input);
+              $this->success []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23726,9 +23726,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter789)
+          foreach ($this->success as $iter798)
           {
-            $xfer += $iter789->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -23935,14 +23935,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size790 = 0;
-            $_etype793 = 0;
-            $xfer += $input->readListBegin($_etype793, $_size790);
-            for ($_i794 = 0; $_i794 < $_size790; ++$_i794)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem795 = null;
-              $xfer += $input->readString($elem795);
-              $this->success []= $elem795;
+              $elem804 = null;
+              $xfer += $input->readString($elem804);
+              $this->success []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23978,9 +23978,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter796)
+          foreach ($this->success as $iter805)
           {
-            $xfer += $output->writeString($iter796);
+            $xfer += $output->writeString($iter805);
           }
         }
         $output->writeListEnd();
@@ -24096,14 +24096,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size797 = 0;
-            $_etype800 = 0;
-            $xfer += $input->readListBegin($_etype800, $_size797);
-            for ($_i801 = 0; $_i801 < $_size797; ++$_i801)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem802 = null;
-              $xfer += $input->readString($elem802);
-              $this->part_vals []= $elem802;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->part_vals []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24148,9 +24148,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter803)
+          foreach ($this->part_vals as $iter812)
           {
-            $xfer += $output->writeString($iter803);
+            $xfer += $output->writeString($iter812);
           }
         }
         $output->writeListEnd();
@@ -24244,15 +24244,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size804 = 0;
-            $_etype807 = 0;
-            $xfer += $input->readListBegin($_etype807, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem809 = null;
-              $elem809 = new \metastore\Partition();
-              $xfer += $elem809->read($input);
-              $this->success []= $elem809;
+              $elem818 = null;
+              $elem818 = new \metastore\Partition();
+              $xfer += $elem818->read($input);
+              $this->success []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24296,9 +24296,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter810)
+          foreach ($this->success as $iter819)
           {
-            $xfer += $iter810->write($output);
+            $xfer += $iter819->write($output);
           }
         }
         $output->writeListEnd();
@@ -24445,14 +24445,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size811 = 0;
-            $_etype814 = 0;
-            $xfer += $input->readListBegin($_etype814, $_size811);
-            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem816 = null;
-              $xfer += $input->readString($elem816);
-              $this->part_vals []= $elem816;
+              $elem825 = null;
+              $xfer += $input->readString($elem825);
+              $this->part_vals []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24476,14 +24476,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size817 = 0;
-            $_etype820 = 0;
-            $xfer += $input->readListBegin($_etype820, $_size817);
-            for ($_i821 = 0; $_i821 < $_size817; ++$_i821)
+            $_size826 = 0;
+            $_etype829 = 0;
+            $xfer += $input->readListBegin($_etype829, $_size826);
+            for ($_i830 = 0; $_i830 < $_size826; ++$_i830)
             {
-              $elem822 = null;
-              $xfer += $input->readString($elem822);
-              $this->group_names []= $elem822;
+              $elem831 = null;
+              $xfer += $input->readString($elem831);
+              $this->group_names []= $elem831;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24521,9 +24521,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter823)
+          foreach ($this->part_vals as $iter832)
           {
-            $xfer += $output->writeString($iter823);
+            $xfer += $output->writeString($iter832);
           }
         }
         $output->writeListEnd();
@@ -24548,9 +24548,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter824)
+          foreach ($this->group_names as $iter833)
           {
-            $xfer += $output->writeString($iter824);
+            $xfer += $output->writeString($iter833);
           }
         }
         $output->writeListEnd();
@@ -24639,15 +24639,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size825 = 0;
-            $_etype828 = 0;
-            $xfer += $input->readListBegin($_etype828, $_size825);
-            for ($_i829 = 0; $_i829 < $_size825; ++$_i829)
+            $_size834 = 0;
+            $_etype837 = 0;
+            $xfer += $input->readListBegin($_etype837, $_size834);
+            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
             {
-              $elem830 = null;
-              $elem830 = new \metastore\Partition();
-              $xfer += $elem830->read($input);
-              $this->success []= $elem830;
+              $elem839 = null;
+              $elem839 = new \metastore\Partition();
+              $xfer += $elem839->read($input);
+              $this->success []= $elem839;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24691,9 +24691,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter831)
+          foreach ($this->success as $iter840)
           {
-            $xfer += $iter831->write($output);
+            $xfer += $iter840->write($output);
           }
         }
         $output->writeListEnd();
@@ -24814,14 +24814,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size832 = 0;
-            $_etype835 = 0;
-            $xfer += $input->readListBegin($_etype835, $_size832);
-            for ($_i836 = 0; $_i836 < $_size832; ++$_i836)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem837 = null;
-              $xfer += $input->readString($elem837);
-              $this->part_vals []= $elem837;
+              $elem846 = null;
+              $xfer += $input->readString($elem846);
+              $this->part_vals []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24866,9 +24866,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter838)
+          foreach ($this->part_vals as $iter847)
           {
-            $xfer += $output->writeString($iter838);
+            $xfer += $output->writeString($iter847);
           }
         }
         $output->writeListEnd();
@@ -24961,14 +24961,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size839 = 0;
-            $_etype842 = 0;
-            $xfer += $input->readListBegin($_etype842, $_size839);
-            for ($_i843 = 0; $_i843 < $_size839; ++$_i843)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem844 = null;
-              $xfer += $input->readString($elem844);
-              $this->success []= $elem844;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->success []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25012,9 +25012,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter845)
+          foreach ($this->success as $iter854)
           {
-            $xfer += $output->writeString($iter845);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -25257,15 +25257,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size846 = 0;
-            $_etype849 = 0;
-            $xfer += $input->readListBegin($_etype849, $_size846);
-            for ($_i850 = 0; $_i850 < $_size846; ++$_i850)
+            $_size855 = 0;
+            $_etype858 = 0;
+            $xfer += $input->readListBegin($_etype858, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $elem851 = null;
-              $elem851 = new \metastore\Partition();
-              $xfer += $elem851->read($input);
-              $this->success []= $elem851;
+              $elem860 = null;
+              $elem860 = new \metastore\Partition();
+              $xfer += $elem860->read($input);
+              $this->success []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25309,9 +25309,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter852)
+          foreach ($this->success as $iter861)
           {
-            $xfer += $iter852->write($output);
+            $xfer += $iter861->write($output);
           }
         }
         $output->writeListEnd();
@@ -25554,15 +25554,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size853 = 0;
-            $_etype856 = 0;
-            $xfer += $input->readListBegin($_etype856, $_size853);
-            for ($_i857 = 0; $_i857 < $_size853; ++$_i857)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem858 = null;
-              $elem858 = new \metastore\PartitionSpec();
-              $xfer += $elem858->read($input);
-              $this->success []= $elem858;
+              $elem867 = null;
+              $elem867 = new \metastore\PartitionSpec();
+              $xfer += $elem867->read($input);
+              $this->success []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25606,9 +25606,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter859)
+          foreach ($this->success as $iter868)
           {
-            $xfer += $iter859->write($output);
+            $xfer += $iter868->write($output);
           }
         }
         $output->writeListEnd();
@@ -26174,14 +26174,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size860 = 0;
-            $_etype863 = 0;
-            $xfer += $input->readListBegin($_etype863, $_size860);
-            for ($_i864 = 0; $_i864 < $_size860; ++$_i864)
+            $_size869 = 0;
+            $_etype872 = 0;
+            $xfer += $input->readListBegin($_etype872, $_size869);
+            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
             {
-              $elem865 = null;
-              $xfer += $input->readString($elem865);
-              $this->names []= $elem865;
+              $elem874 = null;
+              $xfer += $input->readString($elem874);
+              $this->names []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26219,9 +26219,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter866)
+          foreach ($this->names as $iter875)
           {
-            $xfer += $output->writeString($iter866);
+            $xfer += $output->writeString($iter875);
           }
         }
         $output->writeListEnd();
@@ -26310,15 +26310,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size867 = 0;
-            $_etype870 = 0;
-            $xfer += $input->readListBegin($_etype870, $_size867);
-            for ($_i871 = 0; $_i871 < $_size867; ++$_i871)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem872 = null;
-              $elem872 = new \metastore\Partition();
-              $xfer += $elem872->read($input);
-              $this->success []= $elem872;
+              $elem881 = null;
+              $elem881 = new \metastore\Partition();
+              $xfer += $elem881->read($input);
+              $this->success []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26362,9 +26362,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter873)
+          foreach ($this->success as $iter882)
           {
-            $xfer += $iter873->write($output);
+            $xfer += $iter882->write($output);
           }
         }
         $output->writeListEnd();
@@ -26703,15 +26703,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size874 = 0;
-            $_etype877 = 0;
-            $xfer += $input->readListBegin($_etype877, $_size874);
-            for ($_i878 = 0; $_i878 < $_size874; ++$_i878)
+            $_size883 = 0;
+            $_etype886 = 0;
+            $xfer += $input->readListBegin($_etype886, $_size883);
+            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
             {
-              $elem879 = null;
-              $elem879 = new \metastore\Partition();
-              $xfer += $elem879->read($input);
-              $this->new_parts []= $elem879;
+              $elem888 = null;
+              $elem888 = new \metastore\Partition();
+              $xfer += $elem888->read($input);
+              $this->new_parts []= $elem888;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26749,9 +26749,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter880)
+          foreach ($this->new_parts as $iter889)
           {
-            $xfer += $iter880->write($output);
+            $xfer += $iter889->write($output);
           }
         }
         $output->writeListEnd();
@@ -26966,15 +26966,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size881 = 0;
-            $_etype884 = 0;
-            $xfer += $input->readListBegin($_etype884, $_size881);
-            for ($_i885 = 0; $_i885 < $_size881; ++$_i885)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem886 = null;
-              $elem886 = new \metastore\Partition();
-              $xfer += $elem886->read($input);
-              $this->new_parts []= $elem886;
+              $elem895 = null;
+              $elem895 = new \metastore\Partition();
+              $xfer += $elem895->read($input);
+              $this->new_parts []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27020,9 +27020,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter887)
+          foreach ($this->new_parts as $iter896)
           {
-            $xfer += $iter887->write($output);
+            $xfer += $iter896->write($output);
           }
         }
         $output->writeListEnd();
@@ -27500,14 +27500,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size888 = 0;
-            $_etype891 = 0;
-            $xfer += $input->readListBegin($_etype891, $_size888);
-            for ($_i892 = 0; $_i892 < $_size888; ++$_i892)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem893 = null;
-              $xfer += $input->readString($elem893);
-              $this->part_vals []= $elem893;
+              $elem902 = null;
+              $xfer += $input->readString($elem902);
+              $this->part_vals []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27553,9 +27553,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter894)
+          foreach ($this->part_vals as $iter903)
           {
-            $xfer += $output->writeString($iter894);
+            $xfer += $output->writeString($iter903);
           }
         }
         $output->writeListEnd();
@@ -27740,14 +27740,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size895 = 0;
-            $_etype898 = 0;
-            $xfer += $input->readListBegin($_etype898, $_size895);
-            for ($_i899 = 0; $_i899 < $_size895; ++$_i899)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem900 = null;
-              $xfer += $input->readString($elem900);
-              $this->part_vals []= $elem900;
+              $elem909 = null;
+              $xfer += $input->readString($elem909);
+              $this->part_vals []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27782,9 +27782,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter901)
+          foreach ($this->part_vals as $iter910)
           {
-            $xfer += $output->writeString($iter901);
+            $xfer += $output->writeString($iter910);
           }
         }
         $output->writeListEnd();
@@ -28238,14 +28238,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size902 = 0;
-            $_etype905 = 0;
-            $xfer += $input->readListBegin($_etype905, $_size902);
-            for ($_i906 = 0; $_i906 < $_size902; ++$_i906)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem907 = null;
-              $xfer += $input->readString($elem907);
-              $this->success []= $elem907;
+              $elem916 = null;
+              $xfer += $input->readString($elem916);
+              $this->success []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28281,9 +28281,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter908)
+          foreach ($this->success as $iter917)
           {
-            $xfer += $output->writeString($iter908);
+            $xfer += $output->writeString($iter917);
           }
         }
         $output->writeListEnd();
@@ -28443,17 +28443,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size909 = 0;
-            $_ktype910 = 0;
-            $_vtype911 = 0;
-            $xfer += $input->readMapBegin($_ktype910, $_vtype911, $_size909);
-            for ($_i913 = 0; $_i913 < $_size909; ++$_i913)
+            $_size918 = 0;
+            $_ktype919 = 0;
+            $_vtype920 = 0;
+            $xfer += $input->readMapBegin($_ktype919, $_vtype920, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $key914 = '';
-              $val915 = '';
-              $xfer += $input->readString($key914);
-              $xfer += $input->readString($val915);
-              $this->success[$key914] = $val915;
+              $key923 = '';
+              $val924 = '';
+              $xfer += $input->readString($key923);
+              $xfer += $input->readString($val924);
+              $this->success[$key923] = $val924;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28489,10 +28489,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter916 => $viter917)
+          foreach ($this->success as $kiter925 => $viter926)
           {
-            $xfer += $output->writeString($kiter916);
-            $xfer += $output->writeString($viter917);
+            $xfer += $output->writeString($kiter925);
+            $xfer += $output->writeString($viter926);
           }
         }
         $output->writeMapEnd();
@@ -28612,17 +28612,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size918 = 0;
-            $_ktype919 = 0;
-            $_vtype920 = 0;
-            $xfer += $input->readMapBegin($_ktype919, $_vtype920, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size927 = 0;
+            $_ktype928 = 0;
+            $_vtype929 = 0;
+            $xfer += $input->readMapBegin($_ktype928, $_vtype929, $_size927);
+            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
             {
-              $key923 = '';
-              $val924 = '';
-              $xfer += $input->readString($key923);
-              $xfer += $input->readString($val924);
-              $this->part_vals[$key923] = $val924;
+              $key932 = '';
+              $val933 = '';
+              $xfer += $input->readString($key932);
+              $xfer += $input->readString($val933);
+              $this->part_vals[$key932] = $val933;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28667,10 +28667,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter925 => $viter926)
+          foreach ($this->part_vals as $kiter934 => $viter935)
           {
-            $xfer += $output->writeString($kiter925);
-            $xfer += $output->writeString($viter926);
+            $xfer += $output->writeString($kiter934);
+            $xfer += $output->writeString($viter935);
           }
         }
         $output->writeMapEnd();
@@ -28992,17 +28992,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size927 = 0;
-            $_ktype928 = 0;
-            $_vtype929 = 0;
-            $xfer += $input->readMapBegin($_ktype928, $_vtype929, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size936 = 0;
+            $_ktype937 = 0;
+            $_vtype938 = 0;
+            $xfer += $input->readMapBegin($_ktype937, $_vtype938, $_size936);
+            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
             {
-              $key932 = '';
-              $val933 = '';
-              $xfer += $input->readString($key932);
-              $xfer += $input->readString($val933);
-              $this->part_vals[$key932] = $val933;
+              $key941 = '';
+              $val942 = '';
+              $xfer += $input->readString($key941);
+              $xfer += $input->readString($val942);
+              $this->part_vals[$key941] = $val942;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29047,10 +29047,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter934 => $viter935)
+          foreach ($this->part_vals as $kiter943 => $viter944)
           {
-            $xfer += $output->writeString($kiter934);
-            $xfer += $output->writeString($viter935);
+            $xfer += $output->writeString($kiter943);
+            $xfer += $output->writeString($viter944);
           }
         }
         $output->writeMapEnd();
@@ -30524,15 +30524,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size936 = 0;
-            $_etype939 = 0;
-            $xfer += $input->readListBegin($_etype939, $_size936);
-            for ($_i940 = 0; $_i940 < $_size936; ++$_i940)
+            $_size945 = 0;
+            $_etype948 = 0;
+            $xfer += $input->readListBegin($_etype948, $_size945);
+            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
             {
-              $elem941 = null;
-              $elem941 = new \metastore\Index();
-              $xfer += $elem941->read($input);
-              $this->success []= $elem941;
+              $elem950 = null;
+              $elem950 = new \metastore\Index();
+              $xfer += $elem950->read($input);
+              $this->success []= $elem950;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30576,9 +30576,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter942)
+          foreach ($this->success as $iter951)
           {
-            $xfer += $iter942->write($output);
+            $xfer += $iter951->write($output);
           }
         }
         $output->writeListEnd();
@@ -30785,14 +30785,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size943 = 0;
-            $_etype946 = 0;
-            $xfer += $input->readListBegin($_etype946, $_size943);
-            for ($_i947 = 0; $_i947 < $_size943; ++$_i947)
+            $_size952 = 0;
+            $_etype955 = 0;
+            $xfer += $input->readListBegin($_etype955, $_size952);
+            for ($_i956 = 0; $_i956 < $_size952; ++$_i956)
             {
-              $elem948 = null;
-              $xfer += $input->readString($elem948);
-              $this->success []= $elem948;
+              $elem957 = null;
+              $xfer += $input->readString($elem957);
+              $this->success []= $elem957;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30828,9 +30828,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter949)
+          foreach ($this->success as $iter958)
           {
-            $xfer += $output->writeString($iter949);
+            $xfer += $output->writeString($iter958);
           }
         }
         $output->writeListEnd();
@@ -34724,14 +34724,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size950 = 0;
-            $_etype953 = 0;
-            $xfer += $input->readListBegin($_etype953, $_size950);
-            for ($_i954 = 0; $_i954 < $_size950; ++$_i954)
+            $_size959 = 0;
+            $_etype962 = 0;
+            $xfer += $input->readListBegin($_etype962, $_size959);
+            for ($_i963 = 0; $_i963 < $_size959; ++$_i963)
             {
-              $elem955 = null;
-              $xfer += $input->readString($elem955);
-              $this->success []= $elem955;
+              $elem964 = null;
+              $xfer += $input->readString($elem964);
+              $this->success []= $elem964;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34767,9 +34767,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter956)
+          foreach ($this->success as $iter965)
           {
-            $xfer += $output->writeString($iter956);
+            $xfer += $output->writeString($iter965);
           }
         }
         $output->writeListEnd();
@@ -35638,14 +35638,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size957 = 0;
-            $_etype960 = 0;
-            $xfer += $input->readListBegin($_etype960, $_size957);
-            for ($_i961 = 0; $_i961 < $_size957; ++$_i961)
+            $_size966 = 0;
+            $_etype969 = 0;
+            $xfer += $input->readListBegin($_etype969, $_size966);
+            for ($_i970 = 0; $_i970 < $_size966; ++$_i970)
             {
-              $elem962 = null;
-              $xfer += $input->readString($elem962);
-              $this->success []= $elem962;
+              $elem971 = null;
+              $xfer += $input->readString($elem971);
+              $this->success []= $elem971;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35681,9 +35681,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter963)
+          foreach ($this->success as $iter972)
           {
-            $xfer += $output->writeString($iter963);
+            $xfer += $output->writeString($iter972);
           }
         }
         $output->writeListEnd();
@@ -36374,15 +36374,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size964 = 0;
-            $_etype967 = 0;
-            $xfer += $input->readListBegin($_etype967, $_size964);
-            for ($_i968 = 0; $_i968 < $_size964; ++$_i968)
+            $_size973 = 0;
+            $_etype976 = 0;
+            $xfer += $input->readListBegin($_etype976, $_size973);
+            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
             {
-              $elem969 = null;
-              $elem969 = new \metastore\Role();
-              $xfer += $elem969->read($input);
-              $this->success []= $elem969;
+              $elem978 = null;
+              $elem978 = new \metastore\Role();
+              $xfer += $elem978->read($input);
+              $this->success []= $elem978;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36418,9 +36418,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter970)
+          foreach ($this->success as $iter979)
           {
-            $xfer += $iter970->write($output);
+            $xfer += $iter979->write($output);
           }
         }
         $output->writeListEnd();
@@ -37082,14 +37082,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size971 = 0;
-            $_etype974 = 0;
-            $xfer += $input->readListBegin($_etype974, $_size971);
-            for ($_i975 = 0; $_i975 < $_size971; ++$_i975)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem976 = null;
-              $xfer += $input->readString($elem976);
-              $this->group_names []= $elem976;
+              $elem985 = null;
+              $xfer += $input->readString($elem985);
+              $this->group_names []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37130,9 +37130,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter977)
+          foreach ($this->group_names as $iter986)
           {
-            $xfer += $output->writeString($iter977);
+            $xfer += $output->writeString($iter986);
           }
         }
         $output->writeListEnd();
@@ -37440,15 +37440,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size978 = 0;
-            $_etype981 = 0;
-            $xfer += $input->readListBegin($_etype981, $_size978);
-            for ($_i982 = 0; $_i982 < $_size978; ++$_i982)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem983 = null;
-              $elem983 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem983->read($input);
-              $this->success []= $elem983;
+              $elem992 = null;
+              $elem992 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem992->read($input);
+              $this->success []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37484,9 +37484,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter984)
+          foreach ($this->success as $iter993)
           {
-            $xfer += $iter984->write($output);
+            $xfer += $iter993->write($output);
           }
         }
         $output->writeListEnd();
@@ -38118,14 +38118,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size985 = 0;
-            $_etype988 = 0;
-            $xfer += $input->readListBegin($_etype988, $_size985);
-            for ($_i989 = 0; $_i989 < $_size985; ++$_i989)
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $elem990 = null;
-              $xfer += $input->readString($elem990);
-              $this->group_names []= $elem990;
+              $elem999 = null;
+              $xfer += $input->readString($elem999);
+              $this->group_names []= $elem999;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38158,9 +38158,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter991)
+          foreach ($this->group_names as $iter1000)
           {
-            $xfer += $output->writeString($iter991);
+            $xfer += $output->writeString($iter1000);
           }
         }
         $output->writeListEnd();
@@ -38236,14 +38236,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem997 = null;
-              $xfer += $input->readString($elem997);
-              $this->success []= $elem997;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->success []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38279,9 +38279,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter998)
+          foreach ($this->success as $iter1007)
           {
-            $xfer += $output->writeString($iter998);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -39398,14 +39398,14 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size1008 = 0;
+            $_etype1011 = 0;
+            $xfer += $input->readListBegin($_etype1011, $_size1008);
+            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
             {
-              $elem1004 = null;
-              $xfer += $input->readString($elem1004);
-              $this->success []= $elem1004;
+              $elem1013 = null;
+              $xfer += $input->readString($elem1013);
+              $this->success []= $elem1013;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -39433,9 +39433,9 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1005)
+          foreach ($this->success as $iter1014)
           {
-            $xfer += $output->writeString($iter1005);
+            $xfer += $output->writeString($iter1014);
           }
         }
         $output->writeListEnd();
@@ -40074,14 +40074,14 @@ class ThriftHiveMetastore_get_master_keys_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-     

<TRUNCATED>