You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2016/12/19 22:10:02 UTC

[1/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Repository: hive
Updated Branches:
  refs/heads/master 12f5550ca -> bbd99ed60


http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 7927a46..2f1c3cf 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -11144,15 +11144,18 @@ class InsertEventRequestData:
   """
   Attributes:
    - filesAdded
+   - filesAddedChecksum
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.LIST, 'filesAdded', (TType.STRING,None), None, ), # 1
+    (2, TType.LIST, 'filesAddedChecksum', (TType.STRING,None), None, ), # 2
   )
 
-  def __init__(self, filesAdded=None,):
+  def __init__(self, filesAdded=None, filesAddedChecksum=None,):
     self.filesAdded = filesAdded
+    self.filesAddedChecksum = filesAddedChecksum
 
   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:
@@ -11173,6 +11176,16 @@ class InsertEventRequestData:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.LIST:
+          self.filesAddedChecksum = []
+          (_etype501, _size498) = iprot.readListBegin()
+          for _i502 in xrange(_size498):
+            _elem503 = iprot.readString()
+            self.filesAddedChecksum.append(_elem503)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -11186,8 +11199,15 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter498 in self.filesAdded:
-        oprot.writeString(iter498)
+      for iter504 in self.filesAdded:
+        oprot.writeString(iter504)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.filesAddedChecksum is not None:
+      oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 2)
+      oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
+      for iter505 in self.filesAddedChecksum:
+        oprot.writeString(iter505)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11202,6 +11222,7 @@ class InsertEventRequestData:
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.filesAdded)
+    value = (value * 31) ^ hash(self.filesAddedChecksum)
     return value
 
   def __repr__(self):
@@ -11340,10 +11361,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype502, _size499) = iprot.readListBegin()
-          for _i503 in xrange(_size499):
-            _elem504 = iprot.readString()
-            self.partitionVals.append(_elem504)
+          (_etype509, _size506) = iprot.readListBegin()
+          for _i510 in xrange(_size506):
+            _elem511 = iprot.readString()
+            self.partitionVals.append(_elem511)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11376,8 +11397,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter505 in self.partitionVals:
-        oprot.writeString(iter505)
+      for iter512 in self.partitionVals:
+        oprot.writeString(iter512)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11564,12 +11585,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype507, _vtype508, _size506 ) = iprot.readMapBegin()
-          for _i510 in xrange(_size506):
-            _key511 = iprot.readI64()
-            _val512 = MetadataPpdResult()
-            _val512.read(iprot)
-            self.metadata[_key511] = _val512
+          (_ktype514, _vtype515, _size513 ) = iprot.readMapBegin()
+          for _i517 in xrange(_size513):
+            _key518 = iprot.readI64()
+            _val519 = MetadataPpdResult()
+            _val519.read(iprot)
+            self.metadata[_key518] = _val519
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11591,9 +11612,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 kiter513,viter514 in self.metadata.items():
-        oprot.writeI64(kiter513)
-        viter514.write(oprot)
+      for kiter520,viter521 in self.metadata.items():
+        oprot.writeI64(kiter520)
+        viter521.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -11663,10 +11684,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype518, _size515) = iprot.readListBegin()
-          for _i519 in xrange(_size515):
-            _elem520 = iprot.readI64()
-            self.fileIds.append(_elem520)
+          (_etype525, _size522) = iprot.readListBegin()
+          for _i526 in xrange(_size522):
+            _elem527 = iprot.readI64()
+            self.fileIds.append(_elem527)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11698,8 +11719,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter521 in self.fileIds:
-        oprot.writeI64(iter521)
+      for iter528 in self.fileIds:
+        oprot.writeI64(iter528)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -11773,11 +11794,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype523, _vtype524, _size522 ) = iprot.readMapBegin()
-          for _i526 in xrange(_size522):
-            _key527 = iprot.readI64()
-            _val528 = iprot.readString()
-            self.metadata[_key527] = _val528
+          (_ktype530, _vtype531, _size529 ) = iprot.readMapBegin()
+          for _i533 in xrange(_size529):
+            _key534 = iprot.readI64()
+            _val535 = iprot.readString()
+            self.metadata[_key534] = _val535
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -11799,9 +11820,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 kiter529,viter530 in self.metadata.items():
-        oprot.writeI64(kiter529)
-        oprot.writeString(viter530)
+      for kiter536,viter537 in self.metadata.items():
+        oprot.writeI64(kiter536)
+        oprot.writeString(viter537)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -11862,10 +11883,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype534, _size531) = iprot.readListBegin()
-          for _i535 in xrange(_size531):
-            _elem536 = iprot.readI64()
-            self.fileIds.append(_elem536)
+          (_etype541, _size538) = iprot.readListBegin()
+          for _i542 in xrange(_size538):
+            _elem543 = iprot.readI64()
+            self.fileIds.append(_elem543)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11882,8 +11903,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter537 in self.fileIds:
-        oprot.writeI64(iter537)
+      for iter544 in self.fileIds:
+        oprot.writeI64(iter544)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11989,20 +12010,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype541, _size538) = iprot.readListBegin()
-          for _i542 in xrange(_size538):
-            _elem543 = iprot.readI64()
-            self.fileIds.append(_elem543)
+          (_etype548, _size545) = iprot.readListBegin()
+          for _i549 in xrange(_size545):
+            _elem550 = iprot.readI64()
+            self.fileIds.append(_elem550)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype547, _size544) = iprot.readListBegin()
-          for _i548 in xrange(_size544):
-            _elem549 = iprot.readString()
-            self.metadata.append(_elem549)
+          (_etype554, _size551) = iprot.readListBegin()
+          for _i555 in xrange(_size551):
+            _elem556 = iprot.readString()
+            self.metadata.append(_elem556)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12024,15 +12045,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter550 in self.fileIds:
-        oprot.writeI64(iter550)
+      for iter557 in self.fileIds:
+        oprot.writeI64(iter557)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter551 in self.metadata:
-        oprot.writeString(iter551)
+      for iter558 in self.metadata:
+        oprot.writeString(iter558)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -12140,10 +12161,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype555, _size552) = iprot.readListBegin()
-          for _i556 in xrange(_size552):
-            _elem557 = iprot.readI64()
-            self.fileIds.append(_elem557)
+          (_etype562, _size559) = iprot.readListBegin()
+          for _i563 in xrange(_size559):
+            _elem564 = iprot.readI64()
+            self.fileIds.append(_elem564)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12160,8 +12181,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter558 in self.fileIds:
-        oprot.writeI64(iter558)
+      for iter565 in self.fileIds:
+        oprot.writeI64(iter565)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12390,11 +12411,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype562, _size559) = iprot.readListBegin()
-          for _i563 in xrange(_size559):
-            _elem564 = Function()
-            _elem564.read(iprot)
-            self.functions.append(_elem564)
+          (_etype569, _size566) = iprot.readListBegin()
+          for _i570 in xrange(_size566):
+            _elem571 = Function()
+            _elem571.read(iprot)
+            self.functions.append(_elem571)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12411,8 +12432,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter565 in self.functions:
-        iter565.write(oprot)
+      for iter572 in self.functions:
+        iter572.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12464,10 +12485,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype569, _size566) = iprot.readListBegin()
-          for _i570 in xrange(_size566):
-            _elem571 = iprot.readI32()
-            self.values.append(_elem571)
+          (_etype576, _size573) = iprot.readListBegin()
+          for _i577 in xrange(_size573):
+            _elem578 = iprot.readI32()
+            self.values.append(_elem578)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12484,8 +12505,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter572 in self.values:
-        oprot.writeI32(iter572)
+      for iter579 in self.values:
+        oprot.writeI32(iter579)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12714,10 +12735,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype576, _size573) = iprot.readListBegin()
-          for _i577 in xrange(_size573):
-            _elem578 = iprot.readString()
-            self.tblNames.append(_elem578)
+          (_etype583, _size580) = iprot.readListBegin()
+          for _i584 in xrange(_size580):
+            _elem585 = iprot.readString()
+            self.tblNames.append(_elem585)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12744,8 +12765,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter579 in self.tblNames:
-        oprot.writeString(iter579)
+      for iter586 in self.tblNames:
+        oprot.writeString(iter586)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -12805,11 +12826,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype583, _size580) = iprot.readListBegin()
-          for _i584 in xrange(_size580):
-            _elem585 = Table()
-            _elem585.read(iprot)
-            self.tables.append(_elem585)
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = Table()
+            _elem592.read(iprot)
+            self.tables.append(_elem592)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12826,8 +12847,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter586 in self.tables:
-        iter586.write(oprot)
+      for iter593 in self.tables:
+        iter593.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 c82edd6..ebed504 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2502,9 +2502,11 @@ end
 class InsertEventRequestData
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FILESADDED = 1
+  FILESADDEDCHECKSUM = 2
 
   FIELDS = {
-    FILESADDED => {:type => ::Thrift::Types::LIST, :name => 'filesAdded', :element => {:type => ::Thrift::Types::STRING}}
+    FILESADDED => {:type => ::Thrift::Types::LIST, :name => 'filesAdded', :element => {:type => ::Thrift::Types::STRING}},
+    FILESADDEDCHECKSUM => {:type => ::Thrift::Types::LIST, :name => 'filesAddedChecksum', :element => {:type => ::Thrift::Types::STRING, :binary => true}, :optional => true}
   }
 
   def struct_fields; FIELDS; end

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index d0a66b0..2892da3 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -6485,23 +6485,23 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public FireEventResponse fire_listener_event(FireEventRequest rqst) throws TException {
       switch (rqst.getData().getSetField()) {
-        case INSERT_DATA:
-          InsertEvent event = new InsertEvent(rqst.getDbName(), rqst.getTableName(),
-              rqst.getPartitionVals(), rqst.getData().getInsertData().getFilesAdded(),
-              rqst.isSuccessful(), this);
-          for (MetaStoreEventListener transactionalListener : transactionalListeners) {
-            transactionalListener.onInsert(event);
-          }
+      case INSERT_DATA:
+        InsertEvent event =
+            new InsertEvent(rqst.getDbName(), rqst.getTableName(), rqst.getPartitionVals(), rqst
+                .getData().getInsertData(), rqst.isSuccessful(), this);
+        for (MetaStoreEventListener transactionalListener : transactionalListeners) {
+          transactionalListener.onInsert(event);
+        }
 
-          for (MetaStoreEventListener listener : listeners) {
-            listener.onInsert(event);
-          }
+        for (MetaStoreEventListener listener : listeners) {
+          listener.onInsert(event);
+        }
 
-          return new FireEventResponse();
+        return new FireEventResponse();
 
-        default:
-          throw new TException("Event type " + rqst.getData().getSetField().toString() +
-              " not currently supported.");
+      default:
+        throw new TException("Event type " + rqst.getData().getSetField().toString()
+            + " not currently supported.");
       }
 
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
index 102754e..d9a42a7 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
@@ -18,18 +18,16 @@
 
 package org.apache.hadoop.hive.metastore.events;
 
-import org.apache.hadoop.hive.metastore.api.ClientCapabilities;
-
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-
 import org.apache.hadoop.hive.metastore.api.GetTableRequest;
-
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Table;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -40,24 +38,26 @@ public class InsertEvent extends ListenerEvent {
   // we have just the string names, but that's fine for what we need.
   private final String db;
   private final String table;
-  private final Map<String,String> keyValues;
+  private final Map<String, String> keyValues;
   private final List<String> files;
+  private List<ByteBuffer> fileChecksums = new ArrayList<ByteBuffer>();
 
   /**
    *
    * @param db name of the database the table is in
    * @param table name of the table being inserted into
    * @param partVals list of partition values, can be null
+   * @param insertData the inserted files & their checksums
    * @param status status of insert, true = success, false = failure
    * @param handler handler that is firing the event
    */
-  public InsertEvent(String db, String table, List<String> partVals, List<String> files,
-                     boolean status, HMSHandler handler) throws MetaException, NoSuchObjectException {
+  public InsertEvent(String db, String table, List<String> partVals,
+      InsertEventRequestData insertData, boolean status, HMSHandler handler) throws MetaException,
+      NoSuchObjectException {
     super(status, handler);
     this.db = db;
     this.table = table;
-    this.files = files;
-    // TODO: why does this use the handler directly?
+    this.files = insertData.getFilesAdded();
     GetTableRequest req = new GetTableRequest(db, table);
     req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
     Table t = handler.get_table_req(req).getTable();
@@ -67,11 +67,15 @@ public class InsertEvent extends ListenerEvent {
         keyValues.put(t.getPartitionKeys().get(i).getName(), partVals.get(i));
       }
     }
+    if (insertData.isSetFilesAddedChecksum()) {
+      fileChecksums = insertData.getFilesAddedChecksum();
+    }
   }
 
   public String getDb() {
     return db;
   }
+
   /**
    * @return The table.
    */
@@ -82,15 +86,25 @@ public class InsertEvent extends ListenerEvent {
   /**
    * @return List of values for the partition keys.
    */
-  public Map<String,String> getPartitionKeyValues() {
+  public Map<String, String> getPartitionKeyValues() {
     return keyValues;
   }
 
   /**
    * Get list of files created as a result of this DML operation
+   *
    * @return list of new files
    */
   public List<String> getFiles() {
     return files;
   }
+
+  /**
+   * Get a list of file checksums corresponding to the files created (if available)
+   *
+   * @return
+   */
+  public List<ByteBuffer> getFileChecksums() {
+    return fileChecksums;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index adf2fd8..fdb8e80 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -229,13 +230,28 @@ public abstract class MessageFactory {
 
   /**
    * Factory method for building insert message
+   *
    * @param db Name of the database the insert occurred in
    * @param table Name of the table the insert occurred in
-   * @param partVals Partition values for the partition that the insert occurred in, may be null
-   *                 if the insert was done into a non-partitioned table
+   * @param partVals Partition values for the partition that the insert occurred in, may be null if
+   *          the insert was done into a non-partitioned table
    * @param files List of files created as a result of the insert, may be null.
    * @return instance of InsertMessage
    */
   public abstract InsertMessage buildInsertMessage(String db, String table,
-                                                   Map<String,String> partVals, List<String> files);
+      Map<String, String> partVals, List<String> files);
+
+  /**
+   * Factory method for building insert message
+   *
+   * @param db Name of the database the insert occurred in
+   * @param table Name of the table the insert occurred in
+   * @param partVals Partition values for the partition that the insert occurred in, may be null if
+   *          the insert was done into a non-partitioned table
+   * @param files List of files created as a result of the insert, may be null
+   * @param fileChecksums List of checksums corresponding to the files added during insert
+   * @return instance of InsertMessage
+   */
+  public abstract InsertMessage buildInsertMessage(String db, String table,
+      Map<String, String> partVals, List<String> files, List<ByteBuffer> fileChecksums);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
index ef89b17..bd9f9ec 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONInsertMessage.java
@@ -22,11 +22,13 @@ package org.apache.hadoop.hive.metastore.messaging.json;
 import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
 import org.codehaus.jackson.annotate.JsonProperty;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
 /**
- * JSON implementation of DropTableMessage.
+ * JSON implementation of InsertMessage
  */
 public class JSONInsertMessage extends InsertMessage {
 
@@ -40,15 +42,19 @@ public class JSONInsertMessage extends InsertMessage {
   List<String> files;
 
   @JsonProperty
-  Map<String,String> partKeyVals;
+  Map<String, String> partKeyVals;
+
+  @JsonProperty
+  List<byte[]> fileChecksums;
 
   /**
    * Default constructor, needed for Jackson.
    */
-  public JSONInsertMessage() {}
+  public JSONInsertMessage() {
+  }
 
   public JSONInsertMessage(String server, String servicePrincipal, String db, String table,
-                           Map<String,String> partKeyVals, List<String> files, Long timestamp) {
+      Map<String, String> partKeyVals, List<String> files, Long timestamp) {
     this.server = server;
     this.servicePrincipal = servicePrincipal;
     this.db = db;
@@ -59,15 +65,30 @@ public class JSONInsertMessage extends InsertMessage {
     checkValid();
   }
 
+  public JSONInsertMessage(String server, String servicePrincipal, String db, String table,
+      Map<String, String> partKeyVals, List<String> files, List<ByteBuffer> checksums,
+      Long timestamp) {
+    this(server, servicePrincipal, db, table, partKeyVals, files, timestamp);
+    fileChecksums = new ArrayList<byte[]>();
+    for (ByteBuffer checksum : checksums) {
+      byte[] checksumBytes = new byte[checksum.remaining()];
+      checksum.get(checksumBytes);
+      fileChecksums.add(checksumBytes);
+    }
+  }
 
   @Override
-  public String getTable() { return table; }
+  public String getTable() {
+    return table;
+  }
 
   @Override
-  public String getServer() { return server; }
+  public String getServer() {
+    return server;
+  }
 
   @Override
-  public Map<String,String> getPartitionKeyValues() {
+  public Map<String, String> getPartitionKeyValues() {
     return partKeyVals;
   }
 
@@ -77,20 +98,29 @@ public class JSONInsertMessage extends InsertMessage {
   }
 
   @Override
-  public String getServicePrincipal() { return servicePrincipal; }
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
 
   @Override
-  public String getDB() { return db; }
+  public String getDB() {
+    return db;
+  }
 
   @Override
-  public Long getTimestamp() { return timestamp; }
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  public List<byte[]> getFileChecksums() {
+    return fileChecksums;
+  }
 
   @Override
   public String toString() {
     try {
       return JSONMessageDeserializer.mapper.writeValueAsString(this);
-    }
-    catch (Exception exception) {
+    } catch (Exception exception) {
       throw new IllegalArgumentException("Could not serialize: ", exception);
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index 0407210..c57f577 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -19,7 +19,9 @@
 
 package org.apache.hadoop.hive.metastore.messaging.json;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -57,6 +59,7 @@ import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.ObjectNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -168,6 +171,13 @@ public class JSONMessageFactory extends MessageFactory {
         files, now());
   }
 
+  @Override
+  public InsertMessage buildInsertMessage(String db, String table, Map<String, String> partKeyVals,
+      List<String> files, List<ByteBuffer> fileChecksums) {
+    return new JSONInsertMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, db, table, partKeyVals,
+        files, fileChecksums, now());
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }
@@ -255,4 +265,30 @@ public class JSONMessageFactory extends MessageFactory {
     deSerializer.deserialize(indexObj, tableJson, "UTF-8");
     return indexObj;
   }
+
+  /**
+   * Convert a json ArrayNode to an ordered list of Strings
+   *
+   * @param arrayNode: the json array node
+   * @param elements: the list to populate
+   * @return
+   */
+  public static List<String> getAsList(ArrayNode arrayNode, List<String> elements) {
+    Iterator<JsonNode> arrayNodeIterator = arrayNode.iterator();
+    while (arrayNodeIterator.hasNext()) {
+      JsonNode node = arrayNodeIterator.next();
+      elements.add(node.asText());
+    }
+    return elements;
+  }
+
+  public static LinkedHashMap<String, String> getAsMap(ObjectNode objectNode,
+      LinkedHashMap<String, String> hashMap) {
+    Iterator<Map.Entry<String, JsonNode>> objectNodeIterator = objectNode.getFields();
+    while (objectNodeIterator.hasNext()) {
+      Map.Entry<String, JsonNode> objectAsMap = objectNodeIterator.next();
+      hashMap.put(objectAsMap.getKey(), objectAsMap.getValue().asText());
+    }
+    return hashMap;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 f62d5f3..be5a6a9 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
@@ -26,7 +26,8 @@ import static org.apache.hadoop.hive.serde.serdeConstants.LINE_DELIM;
 import static org.apache.hadoop.hive.serde.serdeConstants.MAPKEY_DELIM;
 import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
 import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
-
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -57,6 +58,7 @@ import javax.jdo.JDODataStoreException;
 import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -2336,29 +2338,39 @@ private void constructOneLBLocationMap(FileStatus fSta,
         LOG.debug("Not firing dml insert event as " + tbl.getTableName() + " is temporary");
         return;
       }
-      FireEventRequestData data = new FireEventRequestData();
-      InsertEventRequestData insertData = new InsertEventRequestData();
-      data.setInsertData(insertData);
-      if (newFiles != null && newFiles.size() > 0) {
-        for (Path p : newFiles) {
-          insertData.addToFilesAdded(p.toString());
+      try {
+        FileSystem fileSystem = tbl.getDataLocation().getFileSystem(conf);
+        FireEventRequestData data = new FireEventRequestData();
+        InsertEventRequestData insertData = new InsertEventRequestData();
+        data.setInsertData(insertData);
+        if (newFiles != null && newFiles.size() > 0) {
+          for (Path p : newFiles) {
+            insertData.addToFilesAdded(p.toString());
+            FileChecksum cksum = fileSystem.getFileChecksum(p);
+            // File checksum is not implemented for local filesystem (RawLocalFileSystem)
+            if (cksum != null) {
+              ByteArrayOutputStream baos = new ByteArrayOutputStream();
+              cksum.write(new DataOutputStream(baos));
+              insertData.addToFilesAddedChecksum(ByteBuffer.wrap(baos.toByteArray()));
+            } else {
+              insertData.addToFilesAddedChecksum(ByteBuffer.allocate(0));
+            }
+          }
+        } else {
+          insertData.setFilesAdded(new ArrayList<String>());
         }
-      } else {
-        insertData.setFilesAdded(new ArrayList<String>());
-      }
-      FireEventRequest rqst = new FireEventRequest(true, data);
-      rqst.setDbName(tbl.getDbName());
-      rqst.setTableName(tbl.getTableName());
-      if (partitionSpec != null && partitionSpec.size() > 0) {
-        List<String> partVals = new ArrayList<String>(partitionSpec.size());
-        for (FieldSchema fs : tbl.getPartitionKeys()) {
-          partVals.add(partitionSpec.get(fs.getName()));
+        FireEventRequest rqst = new FireEventRequest(true, data);
+        rqst.setDbName(tbl.getDbName());
+        rqst.setTableName(tbl.getTableName());
+        if (partitionSpec != null && partitionSpec.size() > 0) {
+          List<String> partVals = new ArrayList<String>(partitionSpec.size());
+          for (FieldSchema fs : tbl.getPartitionKeys()) {
+            partVals.add(partitionSpec.get(fs.getName()));
+          }
+          rqst.setPartitionVals(partVals);
         }
-        rqst.setPartitionVals(partVals);
-      }
-      try {
         getMSC().fireListenerEvent(rqst);
-      } catch (TException e) {
+      } catch (IOException | TException e) {
         throw new HiveException(e);
       }
     }
@@ -3019,7 +3031,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
        *
        * I'll leave the below loop for now until a better approach is found.
        */
-    
+
     int counter = 1;
     if (!isRenameAllowed || isBlobStoragePath) {
       while (destFs.exists(destFilePath)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/shims/common/src/main/java/org/apache/hadoop/fs/ProxyLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/fs/ProxyLocalFileSystem.java b/shims/common/src/main/java/org/apache/hadoop/fs/ProxyLocalFileSystem.java
index bd97521..197b965 100644
--- a/shims/common/src/main/java/org/apache/hadoop/fs/ProxyLocalFileSystem.java
+++ b/shims/common/src/main/java/org/apache/hadoop/fs/ProxyLocalFileSystem.java
@@ -18,18 +18,21 @@
 
 package org.apache.hadoop.fs;
 
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.net.URI;
-
+import java.security.MessageDigest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.MD5Hash;
 
 /****************************************************************
  * A Proxy for LocalFileSystem
  *
- * As an example, it serves uri's corresponding to:
- * 'pfile:///' namespace with using a LocalFileSystem
+ * As an example, it serves uri's corresponding to: 'pfile:///' namespace with using a
+ * LocalFileSystem
  *****************************************************************/
 
 public class ProxyLocalFileSystem extends FilterFileSystem {
@@ -46,7 +49,7 @@ public class ProxyLocalFileSystem extends FilterFileSystem {
   }
 
   public ProxyLocalFileSystem(FileSystem fs) {
-    throw new RuntimeException ("Unsupported Constructor");
+    throw new RuntimeException("Unsupported Constructor");
   }
 
   @Override
@@ -59,17 +62,16 @@ public class ProxyLocalFileSystem extends FilterFileSystem {
     if (Shell.WINDOWS) {
       // Replace the encoded backward slash with forward slash
       // Remove the windows drive letter
-      nameUriString = nameUriString.replaceAll("%5C", "/")
-          .replaceFirst("/[c-zC-Z]:", "/")
-          .replaceFirst("^[c-zC-Z]:", "");
+      nameUriString =
+          nameUriString.replaceAll("%5C", "/").replaceFirst("/[c-zC-Z]:", "/")
+              .replaceFirst("^[c-zC-Z]:", "");
       name = URI.create(nameUriString);
     }
 
     String authority = name.getAuthority() != null ? name.getAuthority() : "";
     String proxyUriString = scheme + "://" + authority + "/";
 
-    fs = ShimLoader.getHadoopShims().createProxyFileSystem(
-        localFs, URI.create(proxyUriString));
+    fs = ShimLoader.getHadoopShims().createProxyFileSystem(localFs, URI.create(proxyUriString));
 
     fs.initialize(name, conf);
   }
@@ -78,4 +80,88 @@ public class ProxyLocalFileSystem extends FilterFileSystem {
   public String getScheme() {
     return scheme;
   }
+
+  @Override
+  // For pfile, calculate the checksum for use in testing
+  public FileChecksum getFileChecksum(Path f) throws IOException {
+    if (scheme.equalsIgnoreCase("pfile") && fs.isFile(f)) {
+      return getPFileChecksum(f);
+    }
+    return fs.getFileChecksum(f);
+  }
+
+  private FileChecksum getPFileChecksum(Path f) throws IOException {
+    MessageDigest md5Digest;
+    try {
+      md5Digest = MessageDigest.getInstance("MD5");
+      MD5Hash md5Hash = new MD5Hash(getMD5Checksum(fs.open(f)));
+      return new PFileChecksum(md5Hash, md5Digest.getAlgorithm());
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Calculate MD5 checksum from data in FSDataInputStream
+   * @param fsInputStream
+   * @return byte array with md5 checksum bytes
+   * @throws Exception
+   */
+  static byte[] getMD5Checksum(FSDataInputStream fsInputStream) throws Exception {
+    byte[] buffer = new byte[1024];
+    MessageDigest md5Digest = MessageDigest.getInstance("MD5");
+    int numRead = 0;
+    while (numRead != -1) {
+      numRead = fsInputStream.read(buffer);
+      if (numRead > 0) {
+        md5Digest.update(buffer, 0, numRead);
+      }
+    }
+    fsInputStream.close();
+    return md5Digest.digest();
+  }
+
+  /**
+   * Checksum implementation for PFile uses in testing
+   */
+  public static class PFileChecksum extends FileChecksum {
+    private MD5Hash md5;
+    private String algorithmName;
+
+    public PFileChecksum(MD5Hash md5, String algorithmName) {
+      this.md5 = md5;
+      this.algorithmName = algorithmName;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      md5.write(out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      md5.readFields(in);
+    }
+
+    @Override
+    public String getAlgorithmName() {
+      return algorithmName;
+    }
+
+    @Override
+    public int getLength() {
+      if (md5 != null) {
+        return md5.getDigest().length;
+      }
+      return 0;
+    }
+
+    @Override
+    public byte[] getBytes() {
+      if (md5 != null) {
+        return md5.getDigest();
+      }
+      return new byte[0];
+    }
+  }
 }


[4/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 d41a99c..b7b7da7 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
@@ -29414,13 +29414,13 @@ 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<String>(_list666.size);
-                  String _elem667;
-                  for (int _i668 = 0; _i668 < _list666.size; ++_i668)
+                  org.apache.thrift.protocol.TList _list674 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list674.size);
+                  String _elem675;
+                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
                   {
-                    _elem667 = iprot.readString();
-                    struct.success.add(_elem667);
+                    _elem675 = iprot.readString();
+                    struct.success.add(_elem675);
                   }
                   iprot.readListEnd();
                 }
@@ -29455,9 +29455,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 _iter669 : struct.success)
+            for (String _iter677 : struct.success)
             {
-              oprot.writeString(_iter669);
+              oprot.writeString(_iter677);
             }
             oprot.writeListEnd();
           }
@@ -29496,9 +29496,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter670 : struct.success)
+            for (String _iter678 : struct.success)
             {
-              oprot.writeString(_iter670);
+              oprot.writeString(_iter678);
             }
           }
         }
@@ -29513,13 +29513,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list671.size);
-            String _elem672;
-            for (int _i673 = 0; _i673 < _list671.size; ++_i673)
+            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list679.size);
+            String _elem680;
+            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
             {
-              _elem672 = iprot.readString();
-              struct.success.add(_elem672);
+              _elem680 = iprot.readString();
+              struct.success.add(_elem680);
             }
           }
           struct.setSuccessIsSet(true);
@@ -30173,13 +30173,13 @@ 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<String>(_list674.size);
-                  String _elem675;
-                  for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+                  org.apache.thrift.protocol.TList _list682 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list682.size);
+                  String _elem683;
+                  for (int _i684 = 0; _i684 < _list682.size; ++_i684)
                   {
-                    _elem675 = iprot.readString();
-                    struct.success.add(_elem675);
+                    _elem683 = iprot.readString();
+                    struct.success.add(_elem683);
                   }
                   iprot.readListEnd();
                 }
@@ -30214,9 +30214,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 _iter677 : struct.success)
+            for (String _iter685 : struct.success)
             {
-              oprot.writeString(_iter677);
+              oprot.writeString(_iter685);
             }
             oprot.writeListEnd();
           }
@@ -30255,9 +30255,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter678 : struct.success)
+            for (String _iter686 : struct.success)
             {
-              oprot.writeString(_iter678);
+              oprot.writeString(_iter686);
             }
           }
         }
@@ -30272,13 +30272,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list679 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list679.size);
-            String _elem680;
-            for (int _i681 = 0; _i681 < _list679.size; ++_i681)
+            org.apache.thrift.protocol.TList _list687 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list687.size);
+            String _elem688;
+            for (int _i689 = 0; _i689 < _list687.size; ++_i689)
             {
-              _elem680 = iprot.readString();
-              struct.success.add(_elem680);
+              _elem688 = iprot.readString();
+              struct.success.add(_elem688);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34885,16 +34885,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map682 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map682.size);
-                  String _key683;
-                  Type _val684;
-                  for (int _i685 = 0; _i685 < _map682.size; ++_i685)
+                  org.apache.thrift.protocol.TMap _map690 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map690.size);
+                  String _key691;
+                  Type _val692;
+                  for (int _i693 = 0; _i693 < _map690.size; ++_i693)
                   {
-                    _key683 = iprot.readString();
-                    _val684 = new Type();
-                    _val684.read(iprot);
-                    struct.success.put(_key683, _val684);
+                    _key691 = iprot.readString();
+                    _val692 = new Type();
+                    _val692.read(iprot);
+                    struct.success.put(_key691, _val692);
                   }
                   iprot.readMapEnd();
                 }
@@ -34929,10 +34929,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> _iter686 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter694 : struct.success.entrySet())
             {
-              oprot.writeString(_iter686.getKey());
-              _iter686.getValue().write(oprot);
+              oprot.writeString(_iter694.getKey());
+              _iter694.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -34971,10 +34971,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter687 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter695 : struct.success.entrySet())
             {
-              oprot.writeString(_iter687.getKey());
-              _iter687.getValue().write(oprot);
+              oprot.writeString(_iter695.getKey());
+              _iter695.getValue().write(oprot);
             }
           }
         }
@@ -34989,16 +34989,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map688 = 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*_map688.size);
-            String _key689;
-            Type _val690;
-            for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+            org.apache.thrift.protocol.TMap _map696 = 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*_map696.size);
+            String _key697;
+            Type _val698;
+            for (int _i699 = 0; _i699 < _map696.size; ++_i699)
             {
-              _key689 = iprot.readString();
-              _val690 = new Type();
-              _val690.read(iprot);
-              struct.success.put(_key689, _val690);
+              _key697 = iprot.readString();
+              _val698 = new Type();
+              _val698.read(iprot);
+              struct.success.put(_key697, _val698);
             }
           }
           struct.setSuccessIsSet(true);
@@ -36033,14 +36033,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list700.size);
+                  FieldSchema _elem701;
+                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                   {
-                    _elem693 = new FieldSchema();
-                    _elem693.read(iprot);
-                    struct.success.add(_elem693);
+                    _elem701 = new FieldSchema();
+                    _elem701.read(iprot);
+                    struct.success.add(_elem701);
                   }
                   iprot.readListEnd();
                 }
@@ -36093,9 +36093,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 _iter695 : struct.success)
+            for (FieldSchema _iter703 : struct.success)
             {
-              _iter695.write(oprot);
+              _iter703.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -36150,9 +36150,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter696 : struct.success)
+            for (FieldSchema _iter704 : struct.success)
             {
-              _iter696.write(oprot);
+              _iter704.write(oprot);
             }
           }
         }
@@ -36173,14 +36173,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list705.size);
+            FieldSchema _elem706;
+            for (int _i707 = 0; _i707 < _list705.size; ++_i707)
             {
-              _elem698 = new FieldSchema();
-              _elem698.read(iprot);
-              struct.success.add(_elem698);
+              _elem706 = new FieldSchema();
+              _elem706.read(iprot);
+              struct.success.add(_elem706);
             }
           }
           struct.setSuccessIsSet(true);
@@ -37334,14 +37334,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list700.size);
-                  FieldSchema _elem701;
-                  for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                  org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list708.size);
+                  FieldSchema _elem709;
+                  for (int _i710 = 0; _i710 < _list708.size; ++_i710)
                   {
-                    _elem701 = new FieldSchema();
-                    _elem701.read(iprot);
-                    struct.success.add(_elem701);
+                    _elem709 = new FieldSchema();
+                    _elem709.read(iprot);
+                    struct.success.add(_elem709);
                   }
                   iprot.readListEnd();
                 }
@@ -37394,9 +37394,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 _iter703 : struct.success)
+            for (FieldSchema _iter711 : struct.success)
             {
-              _iter703.write(oprot);
+              _iter711.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -37451,9 +37451,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter704 : struct.success)
+            for (FieldSchema _iter712 : struct.success)
             {
-              _iter704.write(oprot);
+              _iter712.write(oprot);
             }
           }
         }
@@ -37474,14 +37474,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list705.size);
-            FieldSchema _elem706;
-            for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+            org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list713.size);
+            FieldSchema _elem714;
+            for (int _i715 = 0; _i715 < _list713.size; ++_i715)
             {
-              _elem706 = new FieldSchema();
-              _elem706.read(iprot);
-              struct.success.add(_elem706);
+              _elem714 = new FieldSchema();
+              _elem714.read(iprot);
+              struct.success.add(_elem714);
             }
           }
           struct.setSuccessIsSet(true);
@@ -38526,14 +38526,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list708.size);
-                  FieldSchema _elem709;
-                  for (int _i710 = 0; _i710 < _list708.size; ++_i710)
+                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list716.size);
+                  FieldSchema _elem717;
+                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
                   {
-                    _elem709 = new FieldSchema();
-                    _elem709.read(iprot);
-                    struct.success.add(_elem709);
+                    _elem717 = new FieldSchema();
+                    _elem717.read(iprot);
+                    struct.success.add(_elem717);
                   }
                   iprot.readListEnd();
                 }
@@ -38586,9 +38586,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 _iter711 : struct.success)
+            for (FieldSchema _iter719 : struct.success)
             {
-              _iter711.write(oprot);
+              _iter719.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -38643,9 +38643,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter712 : struct.success)
+            for (FieldSchema _iter720 : struct.success)
             {
-              _iter712.write(oprot);
+              _iter720.write(oprot);
             }
           }
         }
@@ -38666,14 +38666,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list713.size);
-            FieldSchema _elem714;
-            for (int _i715 = 0; _i715 < _list713.size; ++_i715)
+            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list721.size);
+            FieldSchema _elem722;
+            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
             {
-              _elem714 = new FieldSchema();
-              _elem714.read(iprot);
-              struct.success.add(_elem714);
+              _elem722 = new FieldSchema();
+              _elem722.read(iprot);
+              struct.success.add(_elem722);
             }
           }
           struct.setSuccessIsSet(true);
@@ -39827,14 +39827,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list716.size);
-                  FieldSchema _elem717;
-                  for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list724.size);
+                  FieldSchema _elem725;
+                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                   {
-                    _elem717 = new FieldSchema();
-                    _elem717.read(iprot);
-                    struct.success.add(_elem717);
+                    _elem725 = new FieldSchema();
+                    _elem725.read(iprot);
+                    struct.success.add(_elem725);
                   }
                   iprot.readListEnd();
                 }
@@ -39887,9 +39887,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 _iter719 : struct.success)
+            for (FieldSchema _iter727 : struct.success)
             {
-              _iter719.write(oprot);
+              _iter727.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -39944,9 +39944,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter720 : struct.success)
+            for (FieldSchema _iter728 : struct.success)
             {
-              _iter720.write(oprot);
+              _iter728.write(oprot);
             }
           }
         }
@@ -39967,14 +39967,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list721.size);
-            FieldSchema _elem722;
-            for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+            org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list729.size);
+            FieldSchema _elem730;
+            for (int _i731 = 0; _i731 < _list729.size; ++_i731)
             {
-              _elem722 = new FieldSchema();
-              _elem722.read(iprot);
-              struct.success.add(_elem722);
+              _elem730 = new FieldSchema();
+              _elem730.read(iprot);
+              struct.success.add(_elem730);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42699,14 +42699,14 @@ public class ThriftHiveMetastore {
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list724.size);
-                  SQLPrimaryKey _elem725;
-                  for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                  org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list732.size);
+                  SQLPrimaryKey _elem733;
+                  for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                   {
-                    _elem725 = new SQLPrimaryKey();
-                    _elem725.read(iprot);
-                    struct.primaryKeys.add(_elem725);
+                    _elem733 = new SQLPrimaryKey();
+                    _elem733.read(iprot);
+                    struct.primaryKeys.add(_elem733);
                   }
                   iprot.readListEnd();
                 }
@@ -42718,14 +42718,14 @@ public class ThriftHiveMetastore {
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list727 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list727.size);
-                  SQLForeignKey _elem728;
-                  for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+                  org.apache.thrift.protocol.TList _list735 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list735.size);
+                  SQLForeignKey _elem736;
+                  for (int _i737 = 0; _i737 < _list735.size; ++_i737)
                   {
-                    _elem728 = new SQLForeignKey();
-                    _elem728.read(iprot);
-                    struct.foreignKeys.add(_elem728);
+                    _elem736 = new SQLForeignKey();
+                    _elem736.read(iprot);
+                    struct.foreignKeys.add(_elem736);
                   }
                   iprot.readListEnd();
                 }
@@ -42756,9 +42756,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 _iter730 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter738 : struct.primaryKeys)
             {
-              _iter730.write(oprot);
+              _iter738.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42768,9 +42768,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 _iter731 : struct.foreignKeys)
+            for (SQLForeignKey _iter739 : struct.foreignKeys)
             {
-              _iter731.write(oprot);
+              _iter739.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -42810,18 +42810,18 @@ public class ThriftHiveMetastore {
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter732 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter740 : struct.primaryKeys)
             {
-              _iter732.write(oprot);
+              _iter740.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter733 : struct.foreignKeys)
+            for (SQLForeignKey _iter741 : struct.foreignKeys)
             {
-              _iter733.write(oprot);
+              _iter741.write(oprot);
             }
           }
         }
@@ -42838,28 +42838,28 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list734 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list734.size);
-            SQLPrimaryKey _elem735;
-            for (int _i736 = 0; _i736 < _list734.size; ++_i736)
+            org.apache.thrift.protocol.TList _list742 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list742.size);
+            SQLPrimaryKey _elem743;
+            for (int _i744 = 0; _i744 < _list742.size; ++_i744)
             {
-              _elem735 = new SQLPrimaryKey();
-              _elem735.read(iprot);
-              struct.primaryKeys.add(_elem735);
+              _elem743 = new SQLPrimaryKey();
+              _elem743.read(iprot);
+              struct.primaryKeys.add(_elem743);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list737.size);
-            SQLForeignKey _elem738;
-            for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+            org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list745.size);
+            SQLForeignKey _elem746;
+            for (int _i747 = 0; _i747 < _list745.size; ++_i747)
             {
-              _elem738 = new SQLForeignKey();
-              _elem738.read(iprot);
-              struct.foreignKeys.add(_elem738);
+              _elem746 = new SQLForeignKey();
+              _elem746.read(iprot);
+              struct.foreignKeys.add(_elem746);
             }
           }
           struct.setForeignKeysIsSet(true);
@@ -49058,13 +49058,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list748.size);
+                  String _elem749;
+                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
                   {
-                    _elem741 = iprot.readString();
-                    struct.success.add(_elem741);
+                    _elem749 = iprot.readString();
+                    struct.success.add(_elem749);
                   }
                   iprot.readListEnd();
                 }
@@ -49099,9 +49099,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 _iter743 : struct.success)
+            for (String _iter751 : struct.success)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter751);
             }
             oprot.writeListEnd();
           }
@@ -49140,9 +49140,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter744 : struct.success)
+            for (String _iter752 : struct.success)
             {
-              oprot.writeString(_iter744);
+              oprot.writeString(_iter752);
             }
           }
         }
@@ -49157,13 +49157,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list753.size);
+            String _elem754;
+            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
             {
-              _elem746 = iprot.readString();
-              struct.success.add(_elem746);
+              _elem754 = iprot.readString();
+              struct.success.add(_elem754);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50137,13 +50137,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list748 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list748.size);
-                  String _elem749;
-                  for (int _i750 = 0; _i750 < _list748.size; ++_i750)
+                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list756.size);
+                  String _elem757;
+                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                   {
-                    _elem749 = iprot.readString();
-                    struct.success.add(_elem749);
+                    _elem757 = iprot.readString();
+                    struct.success.add(_elem757);
                   }
                   iprot.readListEnd();
                 }
@@ -50178,9 +50178,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 _iter751 : struct.success)
+            for (String _iter759 : struct.success)
             {
-              oprot.writeString(_iter751);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -50219,9 +50219,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter752 : struct.success)
+            for (String _iter760 : struct.success)
             {
-              oprot.writeString(_iter752);
+              oprot.writeString(_iter760);
             }
           }
         }
@@ -50236,13 +50236,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list753.size);
-            String _elem754;
-            for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list761.size);
+            String _elem762;
+            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
             {
-              _elem754 = iprot.readString();
-              struct.success.add(_elem754);
+              _elem762 = iprot.readString();
+              struct.success.add(_elem762);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50747,13 +50747,13 @@ public class ThriftHiveMetastore {
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list756.size);
-                  String _elem757;
-                  for (int _i758 = 0; _i758 < _list756.size; ++_i758)
+                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list764.size);
+                  String _elem765;
+                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
                   {
-                    _elem757 = iprot.readString();
-                    struct.tbl_types.add(_elem757);
+                    _elem765 = iprot.readString();
+                    struct.tbl_types.add(_elem765);
                   }
                   iprot.readListEnd();
                 }
@@ -50789,9 +50789,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 _iter759 : struct.tbl_types)
+            for (String _iter767 : struct.tbl_types)
             {
-              oprot.writeString(_iter759);
+              oprot.writeString(_iter767);
             }
             oprot.writeListEnd();
           }
@@ -50834,9 +50834,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter760 : struct.tbl_types)
+            for (String _iter768 : struct.tbl_types)
             {
-              oprot.writeString(_iter760);
+              oprot.writeString(_iter768);
             }
           }
         }
@@ -50856,13 +50856,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list761.size);
-            String _elem762;
-            for (int _i763 = 0; _i763 < _list761.size; ++_i763)
+            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list769.size);
+            String _elem770;
+            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
             {
-              _elem762 = iprot.readString();
-              struct.tbl_types.add(_elem762);
+              _elem770 = iprot.readString();
+              struct.tbl_types.add(_elem770);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -51268,14 +51268,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list764.size);
-                  TableMeta _elem765;
-                  for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list772.size);
+                  TableMeta _elem773;
+                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
                   {
-                    _elem765 = new TableMeta();
-                    _elem765.read(iprot);
-                    struct.success.add(_elem765);
+                    _elem773 = new TableMeta();
+                    _elem773.read(iprot);
+                    struct.success.add(_elem773);
                   }
                   iprot.readListEnd();
                 }
@@ -51310,9 +51310,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 _iter767 : struct.success)
+            for (TableMeta _iter775 : struct.success)
             {
-              _iter767.write(oprot);
+              _iter775.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51351,9 +51351,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter768 : struct.success)
+            for (TableMeta _iter776 : struct.success)
             {
-              _iter768.write(oprot);
+              _iter776.write(oprot);
             }
           }
         }
@@ -51368,14 +51368,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list769.size);
-            TableMeta _elem770;
-            for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list777.size);
+            TableMeta _elem778;
+            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
             {
-              _elem770 = new TableMeta();
-              _elem770.read(iprot);
-              struct.success.add(_elem770);
+              _elem778 = new TableMeta();
+              _elem778.read(iprot);
+              struct.success.add(_elem778);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52141,13 +52141,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list772 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list772.size);
-                  String _elem773;
-                  for (int _i774 = 0; _i774 < _list772.size; ++_i774)
+                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list780.size);
+                  String _elem781;
+                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
                   {
-                    _elem773 = iprot.readString();
-                    struct.success.add(_elem773);
+                    _elem781 = iprot.readString();
+                    struct.success.add(_elem781);
                   }
                   iprot.readListEnd();
                 }
@@ -52182,9 +52182,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 _iter775 : struct.success)
+            for (String _iter783 : struct.success)
             {
-              oprot.writeString(_iter775);
+              oprot.writeString(_iter783);
             }
             oprot.writeListEnd();
           }
@@ -52223,9 +52223,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter776 : struct.success)
+            for (String _iter784 : struct.success)
             {
-              oprot.writeString(_iter776);
+              oprot.writeString(_iter784);
             }
           }
         }
@@ -52240,13 +52240,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list777 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list777.size);
-            String _elem778;
-            for (int _i779 = 0; _i779 < _list777.size; ++_i779)
+            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list785.size);
+            String _elem786;
+            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
             {
-              _elem778 = iprot.readString();
-              struct.success.add(_elem778);
+              _elem786 = iprot.readString();
+              struct.success.add(_elem786);
             }
           }
           struct.setSuccessIsSet(true);
@@ -53699,13 +53699,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list780 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list780.size);
-                  String _elem781;
-                  for (int _i782 = 0; _i782 < _list780.size; ++_i782)
+                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list788.size);
+                  String _elem789;
+                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
                   {
-                    _elem781 = iprot.readString();
-                    struct.tbl_names.add(_elem781);
+                    _elem789 = iprot.readString();
+                    struct.tbl_names.add(_elem789);
                   }
                   iprot.readListEnd();
                 }
@@ -53736,9 +53736,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 _iter783 : struct.tbl_names)
+            for (String _iter791 : struct.tbl_names)
             {
-              oprot.writeString(_iter783);
+              oprot.writeString(_iter791);
             }
             oprot.writeListEnd();
           }
@@ -53775,9 +53775,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter784 : struct.tbl_names)
+            for (String _iter792 : struct.tbl_names)
             {
-              oprot.writeString(_iter784);
+              oprot.writeString(_iter792);
             }
           }
         }
@@ -53793,13 +53793,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list785 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list785.size);
-            String _elem786;
-            for (int _i787 = 0; _i787 < _list785.size; ++_i787)
+            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list793.size);
+            String _elem794;
+            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
             {
-              _elem786 = iprot.readString();
-              struct.tbl_names.add(_elem786);
+              _elem794 = iprot.readString();
+              struct.tbl_names.add(_elem794);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -54124,14 +54124,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list788 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list788.size);
-                  Table _elem789;
-                  for (int _i790 = 0; _i790 < _list788.size; ++_i790)
+                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list796.size);
+                  Table _elem797;
+                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
                   {
-                    _elem789 = new Table();
-                    _elem789.read(iprot);
-                    struct.success.add(_elem789);
+                    _elem797 = new Table();
+                    _elem797.read(iprot);
+                    struct.success.add(_elem797);
                   }
                   iprot.readListEnd();
                 }
@@ -54157,9 +54157,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 _iter791 : struct.success)
+            for (Table _iter799 : struct.success)
             {
-              _iter791.write(oprot);
+              _iter799.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -54190,9 +54190,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter792 : struct.success)
+            for (Table _iter800 : struct.success)
             {
-              _iter792.write(oprot);
+              _iter800.write(oprot);
             }
           }
         }
@@ -54204,14 +54204,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list793 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list793.size);
-            Table _elem794;
-            for (int _i795 = 0; _i795 < _list793.size; ++_i795)
+            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list801.size);
+            Table _elem802;
+            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
             {
-              _elem794 = new Table();
-              _elem794.read(iprot);
-              struct.success.add(_elem794);
+              _elem802 = new Table();
+              _elem802.read(iprot);
+              struct.success.add(_elem802);
             }
           }
           struct.setSuccessIsSet(true);
@@ -57324,13 +57324,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list796 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list796.size);
-                  String _elem797;
-                  for (int _i798 = 0; _i798 < _list796.size; ++_i798)
+                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list804.size);
+                  String _elem805;
+                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
                   {
-                    _elem797 = iprot.readString();
-                    struct.success.add(_elem797);
+                    _elem805 = iprot.readString();
+                    struct.success.add(_elem805);
                   }
                   iprot.readListEnd();
                 }
@@ -57383,9 +57383,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 _iter799 : struct.success)
+            for (String _iter807 : struct.success)
             {
-              oprot.writeString(_iter799);
+              oprot.writeString(_iter807);
             }
             oprot.writeListEnd();
           }
@@ -57440,9 +57440,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter800 : struct.success)
+            for (String _iter808 : struct.success)
             {
-              oprot.writeString(_iter800);
+              oprot.writeString(_iter808);
             }
           }
         }
@@ -57463,13 +57463,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list801 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list801.size);
-            String _elem802;
-            for (int _i803 = 0; _i803 < _list801.size; ++_i803)
+            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list809.size);
+            String _elem810;
+            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
             {
-              _elem802 = iprot.readString();
-              struct.success.add(_elem802);
+              _elem810 = iprot.readString();
+              struct.success.add(_elem810);
             }
           }
           struct.setSuccessIsSet(true);
@@ -63328,14 +63328,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list804 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list804.size);
-                  Partition _elem805;
-                  for (int _i806 = 0; _i806 < _list804.size; ++_i806)
+                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list812.size);
+                  Partition _elem813;
+                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
                   {
-                    _elem805 = new Partition();
-                    _elem805.read(iprot);
-                    struct.new_parts.add(_elem805);
+                    _elem813 = new Partition();
+                    _elem813.read(iprot);
+                    struct.new_parts.add(_elem813);
                   }
                   iprot.readListEnd();
                 }
@@ -63361,9 +63361,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 _iter807 : struct.new_parts)
+            for (Partition _iter815 : struct.new_parts)
             {
-              _iter807.write(oprot);
+              _iter815.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -63394,9 +63394,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter808 : struct.new_parts)
+            for (Partition _iter816 : struct.new_parts)
             {
-              _iter808.write(oprot);
+              _iter816.write(oprot);
             }
           }
         }
@@ -63408,14 +63408,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list809 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list809.size);
-            Partition _elem810;
-            for (int _i811 = 0; _i811 < _list809.size; ++_i811)
+            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list817.size);
+            Partition _elem818;
+            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
             {
-              _elem810 = new Partition();
-              _elem810.read(iprot);
-              struct.new_parts.add(_elem810);
+              _elem818 = new Partition();
+              _elem818.read(iprot);
+              struct.new_parts.add(_elem818);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -64416,14 +64416,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list812 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list812.size);
-                  PartitionSpec _elem813;
-                  for (int _i814 = 0; _i814 < _list812.size; ++_i814)
+                  org.apache.thrift.protocol.TList _list820 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list820.size);
+                  PartitionSpec _elem821;
+                  for (int _i822 = 0; _i822 < _list820.size; ++_i822)
                   {
-                    _elem813 = new PartitionSpec();
-                    _elem813.read(iprot);
-                    struct.new_parts.add(_elem813);
+                    _elem821 = new PartitionSpec();
+                    _elem821.read(iprot);
+                    struct.new_parts.add(_elem821);
                   }
                   iprot.readListEnd();
                 }
@@ -64449,9 +64449,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 _iter815 : struct.new_parts)
+            for (PartitionSpec _iter823 : struct.new_parts)
             {
-              _iter815.write(oprot);
+              _iter823.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -64482,9 +64482,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter816 : struct.new_parts)
+            for (PartitionSpec _iter824 : struct.new_parts)
             {
-              _iter816.write(oprot);
+              _iter824.write(oprot);
             }
           }
         }
@@ -64496,14 +64496,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list817 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list817.size);
-            PartitionSpec _elem818;
-            for (int _i819 = 0; _i819 < _list817.size; ++_i819)
+            org.apache.thrift.protocol.TList _list825 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list825.size);
+            PartitionSpec _elem826;
+            for (int _i827 = 0; _i827 < _list825.size; ++_i827)
             {
-              _elem818 = new PartitionSpec();
-              _elem818.read(iprot);
-              struct.new_parts.add(_elem818);
+              _elem826 = new PartitionSpec();
+              _elem826.read(iprot);
+              struct.new_parts.add(_elem826);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -65679,13 +65679,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  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)
+                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list828.size);
+                  String _elem829;
+                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
                   {
-                    _elem821 = iprot.readString();
-                    struct.part_vals.add(_elem821);
+                    _elem829 = iprot.readString();
+                    struct.part_vals.add(_elem829);
                   }
                   iprot.readListEnd();
                 }
@@ -65721,9 +65721,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 _iter823 : struct.part_vals)
+            for (String _iter831 : struct.part_vals)
             {
-              oprot.writeString(_iter823);
+              oprot.writeString(_iter831);
             }
             oprot.writeListEnd();
           }
@@ -65766,9 +65766,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter824 : struct.part_vals)
+            for (String _iter832 : struct.part_vals)
             {
-              oprot.writeString(_iter824);
+              oprot.writeString(_iter832);
             }
           }
         }
@@ -65788,13 +65788,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            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)
+            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list833.size);
+            String _elem834;
+            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
             {
-              _elem826 = iprot.readString();
-              struct.part_vals.add(_elem826);
+              _elem834 = iprot.readString();
+              struct.part_vals.add(_elem834);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -68103,13 +68103,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list828 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list828.size);
-                  String _elem829;
-                  for (int _i830 = 0; _i830 < _list828.size; ++_i830)
+                  org.apache.thrift.protocol.TList _list836 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list836.size);
+                  String _elem837;
+                  for (int _i838 = 0; _i838 < _list836.size; ++_i838)
                   {
-                    _elem829 = iprot.readString();
-                    struct.part_vals.add(_elem829);
+                    _elem837 = iprot.readString();
+                    struct.part_vals.add(_elem837);
                   }
                   iprot.readListEnd();
                 }
@@ -68154,9 +68154,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 _iter831 : struct.part_vals)
+            for (String _iter839 : struct.part_vals)
             {
-              oprot.writeString(_iter831);
+              oprot.writeString(_iter839);
             }
             oprot.writeListEnd();
           }
@@ -68207,9 +68207,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter832 : struct.part_vals)
+            for (String _iter840 : struct.part_vals)
             {
-              oprot.writeString(_iter832);
+              oprot.writeString(_iter840);
             }
           }
         }
@@ -68232,13 +68232,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list833 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list833.size);
-            String _elem834;
-            for (int _i835 = 0; _i835 < _list833.size; ++_i835)
+            org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list841.size);
+            String _elem842;
+            for (int _i843 = 0; _i843 < _list841.size; ++_i843)
             {
-              _elem834 = iprot.readString();
-              struct.part_vals.add(_elem834);
+              _elem842 = iprot.readString();
+              struct.part_vals.add(_elem842);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -72108,13 +72108,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list836 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list836.size);
-                  String _elem837;
-                  for (int _i838 = 0; _i838 < _list836.size; ++_i838)
+                  org.apache.thrift.protocol.TList _list844 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list844.size);
+                  String _elem845;
+                  for (int _i846 = 0; _i846 < _list844.size; ++_i846)
                   {
-                    _elem837 = iprot.readString();
-                    struct.part_vals.add(_elem837);
+                    _elem845 = iprot.readString();
+                    struct.part_vals.add(_elem845);
                   }
                   iprot.readListEnd();
                 }
@@ -72158,9 +72158,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 _iter839 : struct.part_vals)
+            for (String _iter847 : struct.part_vals)
             {
-              oprot.writeString(_iter839);
+              oprot.writeString(_iter847);
             }
             oprot.writeListEnd();
           }
@@ -72209,9 +72209,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter840 : struct.part_vals)
+            for (String _iter848 : struct.part_vals)
             {
-              oprot.writeString(_iter840);
+              oprot.writeString(_iter848);
             }
           }
         }
@@ -72234,13 +72234,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list841 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list841.size);
-            String _elem842;
-            for (int _i843 = 0; _i843 < _list841.size; ++_i843)
+            org.apache.thrift.protocol.TList _list849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list849.size);
+            String _elem850;
+            for (int _i851 = 0; _i851 < _list849.size; ++_i851)
             {
-              _elem842 = iprot.readString();
-              struct.part_vals.add(_elem842);
+              _elem850 = iprot.readString();
+              struct.part_vals.add(_elem850);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -73479,13 +73479,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list844 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list844.size);
-                  String _elem845;
-                  for (int _i846 = 0; _i846 < _list844.size; ++_i846)
+                  org.apache.thrift.protocol.TList _list852 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list852.size);
+                  String _elem853;
+                  for (int _i854 = 0; _i854 < _list852.size; ++_i854)
                   {
-                    _elem845 = iprot.readString();
-                    struct.part_vals.add(_elem845);
+                    _elem853 = iprot.readString();
+                    struct.part_vals.add(_elem853);
                   }
                   iprot.readListEnd();
                 }
@@ -73538,9 +73538,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 _iter847 : struct.part_vals)
+            for (String _iter855 : struct.part_vals)
             {
-              oprot.writeString(_iter847);
+              oprot.writeString(_iter855);
             }
             oprot.writeListEnd();
           }
@@ -73597,9 +73597,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter848 : struct.part_vals)
+            for (String _iter856 : struct.part_vals)
             {
-              oprot.writeString(_iter848);
+              oprot.writeString(_iter856);
             }
           }
         }
@@ -73625,13 +73625,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list849 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list849.size);
-            String _elem850;
-            for (int _i851 = 0; _i851 < _list849.size; ++_i851)
+            org.apache.thrift.protocol.TList _list857 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list857.size);
+            String _elem858;
+            for (int _i859 = 0; _i859 < _list857.size; ++_i859)
             {
-              _elem850 = iprot.readString();
-              struct.part_vals.add(_elem850);
+              _elem858 = iprot.readString();
+              struct.part_vals.add(_elem858);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -78233,13 +78233,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list852 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list852.size);
-                  String _elem853;
-                  for (int _i854 = 0; _i854 < _list852.size; ++_i854)
+                  org.apache.thrift.protocol.TList _list860 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list860.size);
+                  String _elem861;
+                  for (int _i862 = 0; _i862 < _list860.size; ++_i862)
                   {
-                    _elem853 = iprot.readString();
-                    struct.part_vals.add(_elem853);
+                    _elem861 = iprot.readString();
+                    struct.part_vals.add(_elem861);
                   }
                   iprot.readListEnd();
                 }
@@ -78275,9 +78275,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 _iter855 : struct.part_vals)
+            for (String _iter863 : struct.part_vals)
             {
-              oprot.writeString(_iter855);
+              oprot.writeString(_iter863);
             }
             oprot.writeListEnd();
           }
@@ -78320,9 +78320,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter856 : struct.part_vals)
+            for (String _iter864 : struct.part_vals)
             {
-              oprot.writeString(_iter856);
+              oprot.writeString(_iter864);
             }
           }
         }
@@ -78342,13 +78342,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list857 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list857.size);
-            String _elem858;
-            for (int _i859 = 0; _i859 < _list857.size; ++_i859)
+            org.apache.thrift.protocol.TList _list865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list865.size);
+            String _elem866;
+            for (int _i867 = 0; _i867 < _list865.size; ++_i867)
             {
-              _elem858 = iprot.readString();
-              struct.part_vals.add(_elem858);
+              _elem866 = iprot.readString();
+              struct.part_vals.add(_elem866);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -79566,15 +79566,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map860 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map860.size);
-                  String _key861;
-                  String _val862;
-                  for (int _i863 = 0; _i863 < _map860.size; ++_i863)
+                  org.apache.thrift.protocol.TMap _map868 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map868.size);
+                  String _key869;
+                  String _val870;
+                  for (int _i871 = 0; _i871 < _map868.size; ++_i871)
                   {
-                    _key861 = iprot.readString();
-                    _val862 = iprot.readString();
-                    struct.partitionSpecs.put(_key861, _val862);
+                    _key869 = iprot.readString();
+                    _val870 = iprot.readString();
+                    struct.partitionSpecs.put(_key869, _val870);
                   }
                   iprot.readMapEnd();
                 }
@@ -79632,10 +79632,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> _iter864 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter872 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter864.getKey());
-              oprot.writeString(_iter864.getValue());
+              oprot.writeString(_iter872.getKey());
+              oprot.writeString(_iter872.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -79698,10 +79698,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter865 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter873 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter865.getKey());
-              oprot.writeString(_iter865.getValue());
+              oprot.writeString(_iter873.getKey());
+              oprot.writeString(_iter873.getValue());
             }
           }
         }
@@ -79725,15 +79725,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map866 = 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*_map866.size);
-            String _key867;
-            String _val868;
-            for (int _i869 = 0; _i869 < _map866.size; ++_i869)
+            org.apache.thrift.protocol.TMap _map874 = 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*_map874.size);
+            String _key875;
+            String _val876;
+            for (int _i877 = 0; _i877 < _map874.size; ++_i877)
             {
-              _key867 = iprot.readString();
-              _val868 = iprot.readString();
-              struct.partitionSpecs.put(_key867, _val868);
+              _key875 = iprot.readString();
+              _val876 = iprot.readString();
+              struct.partitionSpecs.put(_key875, _val876);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -81179,15 +81179,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map870 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map870.size);
-                  String _key871;
-                  String _val872;
-                  for (int _i873 = 0; _i873 < _map870.size; ++_i873)
+                  org.apache.thrift.protocol.TMap _map878 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map878.size);
+                  String _key879;
+                  String _val880;
+                  for (int _i881 = 0; _i881 < _map878.size; ++_i881)
                   {
-                    _key871 = iprot.readString();
-                    _val872 = iprot.readString();
-                    struct.partitionSpecs.put(_key871, _val872);
+                    _key879 = iprot.readString();
+                    _val880 = iprot.readString();
+                    struct.partitionSpecs.put(_key879, _val880);
                   }
                   iprot.readMapEnd();
                 }
@@ -81245,10 +81245,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> _iter874 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter882 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter874.getKey());
-              oprot.writeString(_iter874.getValue());
+              oprot.writeString(_iter882.getKey());
+              oprot.writeString(_iter882.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -81311,10 +81311,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter875 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter883 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter875.getKey());
-              oprot.writeString(_iter875.getValue());
+              oprot.writeString(_iter883.getKey());
+              oprot.writeString(_iter883.getValue());
             }
           }
         }
@@ -81338,15 +81338,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map876 = 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*_map876.size);
-            String _key877;
-            String _val878;
-            for (int _i879 = 0; _i879 < _map876.size; ++_i879)
+            org.apache.thrift.protocol.TMap _map884 = 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*_map884.size);
+            String _key885;
+            String _val886;
+            for (int _i887 = 0; _i887 < _map884.size; ++_i887)
             {
-              _key877 = iprot.readString();
-              _val878 = iprot.readString();
-              struct.partitionSpecs.put(_key877, _val878);
+              _key885 = iprot.readString();
+              _val886 = iprot.readString();
+              struct.partitionSpecs.put(_key885, _val886);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -82011,14 +82011,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list880.size);
-                  Partition _elem881;
-                  for (int _i882 = 0; _i882 < _list880.size; ++_i882)
+                  org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list888.size);
+                  Partition _elem889;
+                  for (int _i890 = 0; _i890 < _list888.size; ++_i890)
                   {
-                    _elem881 = new Partition();
-                    _elem881.read(iprot);
-                    struct.success.add(_elem881);
+                    _elem889 = new Partition();
+                    _elem889.read(iprot);
+                    struct.success.add(_elem889);
                   }
                   iprot.readListEnd();
                 }
@@ -82080,9 +82080,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 _iter883 : struct.success)
+            for (Partition _iter891 : struct.success)
             {
-              _iter883.write(oprot);
+              _iter891.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -82145,9 +82145,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter884 : struct.success)
+            for (Partition _iter892 : struct.success)
             {
-              _iter884.write(oprot);
+              _iter892.write(oprot);
             }
           }
         }
@@ -82171,14 +82171,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list885.size);
-            Partition _elem886;
-            for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+            org.apache.thrift.protocol.TList _list893 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list893.size);
+            Partition _elem894;
+            for (int _i895 = 0; _i895 < _list893.size; ++_i895)
             {
-              _elem886 = new Partition();
-              _elem886.read(iprot);
-              struct.success.add(_elem886);
+              _elem894 = new Partition();
+              _elem894.read(iprot);
+              struct.success.add(_elem894);
             }
           }
           struct.setSuccessIsSet(true);
@@ -82877,13 +82877,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list888.size);
-                  String _elem889;
-                  for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                  org.apache.thrift.protocol.TList _list896 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list896.size);
+                  String _elem897;
+                  for (int _i898 = 0; _i898 < _list896.size; ++_i898)
                   {
-                    _elem889 = iprot.readString();
-                    struct.part_vals.add(_elem889);
+                    _elem897 = iprot.readString();
+                    struct.part_vals.add(_elem897);
                   }
                   iprot.readListEnd();
                 }
@@ -82903,13 +82903,13 @@ public class ThriftHiveMetastore {
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list891 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list891.size);
-                  String _elem892;
-                  for (int _i893 = 0; _i893 < _list891.size; ++_i893)
+                  org.apache.thrift.protocol.TList _list899 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list899.size);
+                  String _elem900;
+                  for (int _i901 = 0; _i901 < _list899.size; ++_i901)
                   {
-                    _elem892 = iprot.readString();
-                    struct.group_names.add(_elem892);
+                    _elem900 = iprot.readString();
+                    struct.group_names.add(_elem900);
                   }
                   iprot.readListEnd();
                 }
@@ -82945,9 +82945,9 @@ public class ThriftHiveMetastore {
           oprot.writeFiel

<TRUNCATED>

[7/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 481a2c0..42de24e 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 _size812;
-            ::apache::thrift::protocol::TType _etype815;
-            xfer += iprot->readListBegin(_etype815, _size812);
-            this->success.resize(_size812);
-            uint32_t _i816;
-            for (_i816 = 0; _i816 < _size812; ++_i816)
+            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 += iprot->readString(this->success[_i816]);
+              xfer += iprot->readString(this->success[_i822]);
             }
             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 _iter817;
-      for (_iter817 = this->success.begin(); _iter817 != this->success.end(); ++_iter817)
+      std::vector<std::string> ::const_iterator _iter823;
+      for (_iter823 = this->success.begin(); _iter823 != this->success.end(); ++_iter823)
       {
-        xfer += oprot->writeString((*_iter817));
+        xfer += oprot->writeString((*_iter823));
       }
       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 _size818;
-            ::apache::thrift::protocol::TType _etype821;
-            xfer += iprot->readListBegin(_etype821, _size818);
-            (*(this->success)).resize(_size818);
-            uint32_t _i822;
-            for (_i822 = 0; _i822 < _size818; ++_i822)
+            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 += iprot->readString((*(this->success))[_i822]);
+              xfer += iprot->readString((*(this->success))[_i828]);
             }
             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 _size823;
-            ::apache::thrift::protocol::TType _etype826;
-            xfer += iprot->readListBegin(_etype826, _size823);
-            this->success.resize(_size823);
-            uint32_t _i827;
-            for (_i827 = 0; _i827 < _size823; ++_i827)
+            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 += iprot->readString(this->success[_i827]);
+              xfer += iprot->readString(this->success[_i833]);
             }
             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 _iter828;
-      for (_iter828 = this->success.begin(); _iter828 != this->success.end(); ++_iter828)
+      std::vector<std::string> ::const_iterator _iter834;
+      for (_iter834 = this->success.begin(); _iter834 != this->success.end(); ++_iter834)
       {
-        xfer += oprot->writeString((*_iter828));
+        xfer += oprot->writeString((*_iter834));
       }
       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 _size829;
-            ::apache::thrift::protocol::TType _etype832;
-            xfer += iprot->readListBegin(_etype832, _size829);
-            (*(this->success)).resize(_size829);
-            uint32_t _i833;
-            for (_i833 = 0; _i833 < _size829; ++_i833)
+            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 += iprot->readString((*(this->success))[_i833]);
+              xfer += iprot->readString((*(this->success))[_i839]);
             }
             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 _size834;
-            ::apache::thrift::protocol::TType _ktype835;
-            ::apache::thrift::protocol::TType _vtype836;
-            xfer += iprot->readMapBegin(_ktype835, _vtype836, _size834);
-            uint32_t _i838;
-            for (_i838 = 0; _i838 < _size834; ++_i838)
+            uint32_t _size840;
+            ::apache::thrift::protocol::TType _ktype841;
+            ::apache::thrift::protocol::TType _vtype842;
+            xfer += iprot->readMapBegin(_ktype841, _vtype842, _size840);
+            uint32_t _i844;
+            for (_i844 = 0; _i844 < _size840; ++_i844)
             {
-              std::string _key839;
-              xfer += iprot->readString(_key839);
-              Type& _val840 = this->success[_key839];
-              xfer += _val840.read(iprot);
+              std::string _key845;
+              xfer += iprot->readString(_key845);
+              Type& _val846 = this->success[_key845];
+              xfer += _val846.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 _iter841;
-      for (_iter841 = this->success.begin(); _iter841 != this->success.end(); ++_iter841)
+      std::map<std::string, Type> ::const_iterator _iter847;
+      for (_iter847 = this->success.begin(); _iter847 != this->success.end(); ++_iter847)
       {
-        xfer += oprot->writeString(_iter841->first);
-        xfer += _iter841->second.write(oprot);
+        xfer += oprot->writeString(_iter847->first);
+        xfer += _iter847->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 _size842;
-            ::apache::thrift::protocol::TType _ktype843;
-            ::apache::thrift::protocol::TType _vtype844;
-            xfer += iprot->readMapBegin(_ktype843, _vtype844, _size842);
-            uint32_t _i846;
-            for (_i846 = 0; _i846 < _size842; ++_i846)
+            uint32_t _size848;
+            ::apache::thrift::protocol::TType _ktype849;
+            ::apache::thrift::protocol::TType _vtype850;
+            xfer += iprot->readMapBegin(_ktype849, _vtype850, _size848);
+            uint32_t _i852;
+            for (_i852 = 0; _i852 < _size848; ++_i852)
             {
-              std::string _key847;
-              xfer += iprot->readString(_key847);
-              Type& _val848 = (*(this->success))[_key847];
-              xfer += _val848.read(iprot);
+              std::string _key853;
+              xfer += iprot->readString(_key853);
+              Type& _val854 = (*(this->success))[_key853];
+              xfer += _val854.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 _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 _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            this->success.resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += this->success[_i853].read(iprot);
+              xfer += this->success[_i859].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 _iter854;
-      for (_iter854 = this->success.begin(); _iter854 != this->success.end(); ++_iter854)
+      std::vector<FieldSchema> ::const_iterator _iter860;
+      for (_iter860 = this->success.begin(); _iter860 != this->success.end(); ++_iter860)
       {
-        xfer += (*_iter854).write(oprot);
+        xfer += (*_iter860).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 _size855;
-            ::apache::thrift::protocol::TType _etype858;
-            xfer += iprot->readListBegin(_etype858, _size855);
-            (*(this->success)).resize(_size855);
-            uint32_t _i859;
-            for (_i859 = 0; _i859 < _size855; ++_i859)
+            uint32_t _size861;
+            ::apache::thrift::protocol::TType _etype864;
+            xfer += iprot->readListBegin(_etype864, _size861);
+            (*(this->success)).resize(_size861);
+            uint32_t _i865;
+            for (_i865 = 0; _i865 < _size861; ++_i865)
             {
-              xfer += (*(this->success))[_i859].read(iprot);
+              xfer += (*(this->success))[_i865].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 _size860;
-            ::apache::thrift::protocol::TType _etype863;
-            xfer += iprot->readListBegin(_etype863, _size860);
-            this->success.resize(_size860);
-            uint32_t _i864;
-            for (_i864 = 0; _i864 < _size860; ++_i864)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            this->success.resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += this->success[_i864].read(iprot);
+              xfer += this->success[_i870].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 _iter865;
-      for (_iter865 = this->success.begin(); _iter865 != this->success.end(); ++_iter865)
+      std::vector<FieldSchema> ::const_iterator _iter871;
+      for (_iter871 = this->success.begin(); _iter871 != this->success.end(); ++_iter871)
       {
-        xfer += (*_iter865).write(oprot);
+        xfer += (*_iter871).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 _size866;
-            ::apache::thrift::protocol::TType _etype869;
-            xfer += iprot->readListBegin(_etype869, _size866);
-            (*(this->success)).resize(_size866);
-            uint32_t _i870;
-            for (_i870 = 0; _i870 < _size866; ++_i870)
+            uint32_t _size872;
+            ::apache::thrift::protocol::TType _etype875;
+            xfer += iprot->readListBegin(_etype875, _size872);
+            (*(this->success)).resize(_size872);
+            uint32_t _i876;
+            for (_i876 = 0; _i876 < _size872; ++_i876)
             {
-              xfer += (*(this->success))[_i870].read(iprot);
+              xfer += (*(this->success))[_i876].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 _size871;
-            ::apache::thrift::protocol::TType _etype874;
-            xfer += iprot->readListBegin(_etype874, _size871);
-            this->success.resize(_size871);
-            uint32_t _i875;
-            for (_i875 = 0; _i875 < _size871; ++_i875)
+            uint32_t _size877;
+            ::apache::thrift::protocol::TType _etype880;
+            xfer += iprot->readListBegin(_etype880, _size877);
+            this->success.resize(_size877);
+            uint32_t _i881;
+            for (_i881 = 0; _i881 < _size877; ++_i881)
             {
-              xfer += this->success[_i875].read(iprot);
+              xfer += this->success[_i881].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 _iter876;
-      for (_iter876 = this->success.begin(); _iter876 != this->success.end(); ++_iter876)
+      std::vector<FieldSchema> ::const_iterator _iter882;
+      for (_iter882 = this->success.begin(); _iter882 != this->success.end(); ++_iter882)
       {
-        xfer += (*_iter876).write(oprot);
+        xfer += (*_iter882).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 _size877;
-            ::apache::thrift::protocol::TType _etype880;
-            xfer += iprot->readListBegin(_etype880, _size877);
-            (*(this->success)).resize(_size877);
-            uint32_t _i881;
-            for (_i881 = 0; _i881 < _size877; ++_i881)
+            uint32_t _size883;
+            ::apache::thrift::protocol::TType _etype886;
+            xfer += iprot->readListBegin(_etype886, _size883);
+            (*(this->success)).resize(_size883);
+            uint32_t _i887;
+            for (_i887 = 0; _i887 < _size883; ++_i887)
             {
-              xfer += (*(this->success))[_i881].read(iprot);
+              xfer += (*(this->success))[_i887].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 _size882;
-            ::apache::thrift::protocol::TType _etype885;
-            xfer += iprot->readListBegin(_etype885, _size882);
-            this->success.resize(_size882);
-            uint32_t _i886;
-            for (_i886 = 0; _i886 < _size882; ++_i886)
+            uint32_t _size888;
+            ::apache::thrift::protocol::TType _etype891;
+            xfer += iprot->readListBegin(_etype891, _size888);
+            this->success.resize(_size888);
+            uint32_t _i892;
+            for (_i892 = 0; _i892 < _size888; ++_i892)
             {
-              xfer += this->success[_i886].read(iprot);
+              xfer += this->success[_i892].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 _iter887;
-      for (_iter887 = this->success.begin(); _iter887 != this->success.end(); ++_iter887)
+      std::vector<FieldSchema> ::const_iterator _iter893;
+      for (_iter893 = this->success.begin(); _iter893 != this->success.end(); ++_iter893)
       {
-        xfer += (*_iter887).write(oprot);
+        xfer += (*_iter893).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 _size888;
-            ::apache::thrift::protocol::TType _etype891;
-            xfer += iprot->readListBegin(_etype891, _size888);
-            (*(this->success)).resize(_size888);
-            uint32_t _i892;
-            for (_i892 = 0; _i892 < _size888; ++_i892)
+            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))[_i892].read(iprot);
+              xfer += (*(this->success))[_i898].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 _size893;
-            ::apache::thrift::protocol::TType _etype896;
-            xfer += iprot->readListBegin(_etype896, _size893);
-            this->primaryKeys.resize(_size893);
-            uint32_t _i897;
-            for (_i897 = 0; _i897 < _size893; ++_i897)
+            uint32_t _size899;
+            ::apache::thrift::protocol::TType _etype902;
+            xfer += iprot->readListBegin(_etype902, _size899);
+            this->primaryKeys.resize(_size899);
+            uint32_t _i903;
+            for (_i903 = 0; _i903 < _size899; ++_i903)
             {
-              xfer += this->primaryKeys[_i897].read(iprot);
+              xfer += this->primaryKeys[_i903].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 _size898;
-            ::apache::thrift::protocol::TType _etype901;
-            xfer += iprot->readListBegin(_etype901, _size898);
-            this->foreignKeys.resize(_size898);
-            uint32_t _i902;
-            for (_i902 = 0; _i902 < _size898; ++_i902)
+            uint32_t _size904;
+            ::apache::thrift::protocol::TType _etype907;
+            xfer += iprot->readListBegin(_etype907, _size904);
+            this->foreignKeys.resize(_size904);
+            uint32_t _i908;
+            for (_i908 = 0; _i908 < _size904; ++_i908)
             {
-              xfer += this->foreignKeys[_i902].read(iprot);
+              xfer += this->foreignKeys[_i908].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 _iter903;
-    for (_iter903 = this->primaryKeys.begin(); _iter903 != this->primaryKeys.end(); ++_iter903)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter909;
+    for (_iter909 = this->primaryKeys.begin(); _iter909 != this->primaryKeys.end(); ++_iter909)
     {
-      xfer += (*_iter903).write(oprot);
+      xfer += (*_iter909).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 _iter904;
-    for (_iter904 = this->foreignKeys.begin(); _iter904 != this->foreignKeys.end(); ++_iter904)
+    std::vector<SQLForeignKey> ::const_iterator _iter910;
+    for (_iter910 = this->foreignKeys.begin(); _iter910 != this->foreignKeys.end(); ++_iter910)
     {
-      xfer += (*_iter904).write(oprot);
+      xfer += (*_iter910).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 _iter905;
-    for (_iter905 = (*(this->primaryKeys)).begin(); _iter905 != (*(this->primaryKeys)).end(); ++_iter905)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter911;
+    for (_iter911 = (*(this->primaryKeys)).begin(); _iter911 != (*(this->primaryKeys)).end(); ++_iter911)
     {
-      xfer += (*_iter905).write(oprot);
+      xfer += (*_iter911).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 _iter906;
-    for (_iter906 = (*(this->foreignKeys)).begin(); _iter906 != (*(this->foreignKeys)).end(); ++_iter906)
+    std::vector<SQLForeignKey> ::const_iterator _iter912;
+    for (_iter912 = (*(this->foreignKeys)).begin(); _iter912 != (*(this->foreignKeys)).end(); ++_iter912)
     {
-      xfer += (*_iter906).write(oprot);
+      xfer += (*_iter912).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 _size907;
-            ::apache::thrift::protocol::TType _etype910;
-            xfer += iprot->readListBegin(_etype910, _size907);
-            this->success.resize(_size907);
-            uint32_t _i911;
-            for (_i911 = 0; _i911 < _size907; ++_i911)
+            uint32_t _size913;
+            ::apache::thrift::protocol::TType _etype916;
+            xfer += iprot->readListBegin(_etype916, _size913);
+            this->success.resize(_size913);
+            uint32_t _i917;
+            for (_i917 = 0; _i917 < _size913; ++_i917)
             {
-              xfer += iprot->readString(this->success[_i911]);
+              xfer += iprot->readString(this->success[_i917]);
             }
             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 _iter912;
-      for (_iter912 = this->success.begin(); _iter912 != this->success.end(); ++_iter912)
+      std::vector<std::string> ::const_iterator _iter918;
+      for (_iter918 = this->success.begin(); _iter918 != this->success.end(); ++_iter918)
       {
-        xfer += oprot->writeString((*_iter912));
+        xfer += oprot->writeString((*_iter918));
       }
       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 _size913;
-            ::apache::thrift::protocol::TType _etype916;
-            xfer += iprot->readListBegin(_etype916, _size913);
-            (*(this->success)).resize(_size913);
-            uint32_t _i917;
-            for (_i917 = 0; _i917 < _size913; ++_i917)
+            uint32_t _size919;
+            ::apache::thrift::protocol::TType _etype922;
+            xfer += iprot->readListBegin(_etype922, _size919);
+            (*(this->success)).resize(_size919);
+            uint32_t _i923;
+            for (_i923 = 0; _i923 < _size919; ++_i923)
             {
-              xfer += iprot->readString((*(this->success))[_i917]);
+              xfer += iprot->readString((*(this->success))[_i923]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6326,14 +6326,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size918;
-            ::apache::thrift::protocol::TType _etype921;
-            xfer += iprot->readListBegin(_etype921, _size918);
-            this->success.resize(_size918);
-            uint32_t _i922;
-            for (_i922 = 0; _i922 < _size918; ++_i922)
+            uint32_t _size924;
+            ::apache::thrift::protocol::TType _etype927;
+            xfer += iprot->readListBegin(_etype927, _size924);
+            this->success.resize(_size924);
+            uint32_t _i928;
+            for (_i928 = 0; _i928 < _size924; ++_i928)
             {
-              xfer += iprot->readString(this->success[_i922]);
+              xfer += iprot->readString(this->success[_i928]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6372,10 +6372,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter923;
-      for (_iter923 = this->success.begin(); _iter923 != this->success.end(); ++_iter923)
+      std::vector<std::string> ::const_iterator _iter929;
+      for (_iter929 = this->success.begin(); _iter929 != this->success.end(); ++_iter929)
       {
-        xfer += oprot->writeString((*_iter923));
+        xfer += oprot->writeString((*_iter929));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6420,14 +6420,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _etype927;
-            xfer += iprot->readListBegin(_etype927, _size924);
-            (*(this->success)).resize(_size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size930;
+            ::apache::thrift::protocol::TType _etype933;
+            xfer += iprot->readListBegin(_etype933, _size930);
+            (*(this->success)).resize(_size930);
+            uint32_t _i934;
+            for (_i934 = 0; _i934 < _size930; ++_i934)
             {
-              xfer += iprot->readString((*(this->success))[_i928]);
+              xfer += iprot->readString((*(this->success))[_i934]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6502,14 +6502,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 _size929;
-            ::apache::thrift::protocol::TType _etype932;
-            xfer += iprot->readListBegin(_etype932, _size929);
-            this->tbl_types.resize(_size929);
-            uint32_t _i933;
-            for (_i933 = 0; _i933 < _size929; ++_i933)
+            uint32_t _size935;
+            ::apache::thrift::protocol::TType _etype938;
+            xfer += iprot->readListBegin(_etype938, _size935);
+            this->tbl_types.resize(_size935);
+            uint32_t _i939;
+            for (_i939 = 0; _i939 < _size935; ++_i939)
             {
-              xfer += iprot->readString(this->tbl_types[_i933]);
+              xfer += iprot->readString(this->tbl_types[_i939]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6546,10 +6546,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 _iter934;
-    for (_iter934 = this->tbl_types.begin(); _iter934 != this->tbl_types.end(); ++_iter934)
+    std::vector<std::string> ::const_iterator _iter940;
+    for (_iter940 = this->tbl_types.begin(); _iter940 != this->tbl_types.end(); ++_iter940)
     {
-      xfer += oprot->writeString((*_iter934));
+      xfer += oprot->writeString((*_iter940));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6581,10 +6581,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 _iter935;
-    for (_iter935 = (*(this->tbl_types)).begin(); _iter935 != (*(this->tbl_types)).end(); ++_iter935)
+    std::vector<std::string> ::const_iterator _iter941;
+    for (_iter941 = (*(this->tbl_types)).begin(); _iter941 != (*(this->tbl_types)).end(); ++_iter941)
     {
-      xfer += oprot->writeString((*_iter935));
+      xfer += oprot->writeString((*_iter941));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6625,14 +6625,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size936;
-            ::apache::thrift::protocol::TType _etype939;
-            xfer += iprot->readListBegin(_etype939, _size936);
-            this->success.resize(_size936);
-            uint32_t _i940;
-            for (_i940 = 0; _i940 < _size936; ++_i940)
+            uint32_t _size942;
+            ::apache::thrift::protocol::TType _etype945;
+            xfer += iprot->readListBegin(_etype945, _size942);
+            this->success.resize(_size942);
+            uint32_t _i946;
+            for (_i946 = 0; _i946 < _size942; ++_i946)
             {
-              xfer += this->success[_i940].read(iprot);
+              xfer += this->success[_i946].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6671,10 +6671,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 _iter941;
-      for (_iter941 = this->success.begin(); _iter941 != this->success.end(); ++_iter941)
+      std::vector<TableMeta> ::const_iterator _iter947;
+      for (_iter947 = this->success.begin(); _iter947 != this->success.end(); ++_iter947)
       {
-        xfer += (*_iter941).write(oprot);
+        xfer += (*_iter947).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6719,14 +6719,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size942;
-            ::apache::thrift::protocol::TType _etype945;
-            xfer += iprot->readListBegin(_etype945, _size942);
-            (*(this->success)).resize(_size942);
-            uint32_t _i946;
-            for (_i946 = 0; _i946 < _size942; ++_i946)
+            uint32_t _size948;
+            ::apache::thrift::protocol::TType _etype951;
+            xfer += iprot->readListBegin(_etype951, _size948);
+            (*(this->success)).resize(_size948);
+            uint32_t _i952;
+            for (_i952 = 0; _i952 < _size948; ++_i952)
             {
-              xfer += (*(this->success))[_i946].read(iprot);
+              xfer += (*(this->success))[_i952].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6864,14 +6864,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size947;
-            ::apache::thrift::protocol::TType _etype950;
-            xfer += iprot->readListBegin(_etype950, _size947);
-            this->success.resize(_size947);
-            uint32_t _i951;
-            for (_i951 = 0; _i951 < _size947; ++_i951)
+            uint32_t _size953;
+            ::apache::thrift::protocol::TType _etype956;
+            xfer += iprot->readListBegin(_etype956, _size953);
+            this->success.resize(_size953);
+            uint32_t _i957;
+            for (_i957 = 0; _i957 < _size953; ++_i957)
             {
-              xfer += iprot->readString(this->success[_i951]);
+              xfer += iprot->readString(this->success[_i957]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6910,10 +6910,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 _iter952;
-      for (_iter952 = this->success.begin(); _iter952 != this->success.end(); ++_iter952)
+      std::vector<std::string> ::const_iterator _iter958;
+      for (_iter958 = this->success.begin(); _iter958 != this->success.end(); ++_iter958)
       {
-        xfer += oprot->writeString((*_iter952));
+        xfer += oprot->writeString((*_iter958));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6958,14 +6958,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size953;
-            ::apache::thrift::protocol::TType _etype956;
-            xfer += iprot->readListBegin(_etype956, _size953);
-            (*(this->success)).resize(_size953);
-            uint32_t _i957;
-            for (_i957 = 0; _i957 < _size953; ++_i957)
+            uint32_t _size959;
+            ::apache::thrift::protocol::TType _etype962;
+            xfer += iprot->readListBegin(_etype962, _size959);
+            (*(this->success)).resize(_size959);
+            uint32_t _i963;
+            for (_i963 = 0; _i963 < _size959; ++_i963)
             {
-              xfer += iprot->readString((*(this->success))[_i957]);
+              xfer += iprot->readString((*(this->success))[_i963]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7275,14 +7275,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 _size958;
-            ::apache::thrift::protocol::TType _etype961;
-            xfer += iprot->readListBegin(_etype961, _size958);
-            this->tbl_names.resize(_size958);
-            uint32_t _i962;
-            for (_i962 = 0; _i962 < _size958; ++_i962)
+            uint32_t _size964;
+            ::apache::thrift::protocol::TType _etype967;
+            xfer += iprot->readListBegin(_etype967, _size964);
+            this->tbl_names.resize(_size964);
+            uint32_t _i968;
+            for (_i968 = 0; _i968 < _size964; ++_i968)
             {
-              xfer += iprot->readString(this->tbl_names[_i962]);
+              xfer += iprot->readString(this->tbl_names[_i968]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7315,10 +7315,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 _iter963;
-    for (_iter963 = this->tbl_names.begin(); _iter963 != this->tbl_names.end(); ++_iter963)
+    std::vector<std::string> ::const_iterator _iter969;
+    for (_iter969 = this->tbl_names.begin(); _iter969 != this->tbl_names.end(); ++_iter969)
     {
-      xfer += oprot->writeString((*_iter963));
+      xfer += oprot->writeString((*_iter969));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7346,10 +7346,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 _iter964;
-    for (_iter964 = (*(this->tbl_names)).begin(); _iter964 != (*(this->tbl_names)).end(); ++_iter964)
+    std::vector<std::string> ::const_iterator _iter970;
+    for (_iter970 = (*(this->tbl_names)).begin(); _iter970 != (*(this->tbl_names)).end(); ++_iter970)
     {
-      xfer += oprot->writeString((*_iter964));
+      xfer += oprot->writeString((*_iter970));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7390,14 +7390,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 _size965;
-            ::apache::thrift::protocol::TType _etype968;
-            xfer += iprot->readListBegin(_etype968, _size965);
-            this->success.resize(_size965);
-            uint32_t _i969;
-            for (_i969 = 0; _i969 < _size965; ++_i969)
+            uint32_t _size971;
+            ::apache::thrift::protocol::TType _etype974;
+            xfer += iprot->readListBegin(_etype974, _size971);
+            this->success.resize(_size971);
+            uint32_t _i975;
+            for (_i975 = 0; _i975 < _size971; ++_i975)
             {
-              xfer += this->success[_i969].read(iprot);
+              xfer += this->success[_i975].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7428,10 +7428,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 _iter970;
-      for (_iter970 = this->success.begin(); _iter970 != this->success.end(); ++_iter970)
+      std::vector<Table> ::const_iterator _iter976;
+      for (_iter976 = this->success.begin(); _iter976 != this->success.end(); ++_iter976)
       {
-        xfer += (*_iter970).write(oprot);
+        xfer += (*_iter976).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7472,14 +7472,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 _size971;
-            ::apache::thrift::protocol::TType _etype974;
-            xfer += iprot->readListBegin(_etype974, _size971);
-            (*(this->success)).resize(_size971);
-            uint32_t _i975;
-            for (_i975 = 0; _i975 < _size971; ++_i975)
+            uint32_t _size977;
+            ::apache::thrift::protocol::TType _etype980;
+            xfer += iprot->readListBegin(_etype980, _size977);
+            (*(this->success)).resize(_size977);
+            uint32_t _i981;
+            for (_i981 = 0; _i981 < _size977; ++_i981)
             {
-              xfer += (*(this->success))[_i975].read(iprot);
+              xfer += (*(this->success))[_i981].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8115,14 +8115,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 _size976;
-            ::apache::thrift::protocol::TType _etype979;
-            xfer += iprot->readListBegin(_etype979, _size976);
-            this->success.resize(_size976);
-            uint32_t _i980;
-            for (_i980 = 0; _i980 < _size976; ++_i980)
+            uint32_t _size982;
+            ::apache::thrift::protocol::TType _etype985;
+            xfer += iprot->readListBegin(_etype985, _size982);
+            this->success.resize(_size982);
+            uint32_t _i986;
+            for (_i986 = 0; _i986 < _size982; ++_i986)
             {
-              xfer += iprot->readString(this->success[_i980]);
+              xfer += iprot->readString(this->success[_i986]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8177,10 +8177,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 _iter981;
-      for (_iter981 = this->success.begin(); _iter981 != this->success.end(); ++_iter981)
+      std::vector<std::string> ::const_iterator _iter987;
+      for (_iter987 = this->success.begin(); _iter987 != this->success.end(); ++_iter987)
       {
-        xfer += oprot->writeString((*_iter981));
+        xfer += oprot->writeString((*_iter987));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8233,14 +8233,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 _size982;
-            ::apache::thrift::protocol::TType _etype985;
-            xfer += iprot->readListBegin(_etype985, _size982);
-            (*(this->success)).resize(_size982);
-            uint32_t _i986;
-            for (_i986 = 0; _i986 < _size982; ++_i986)
+            uint32_t _size988;
+            ::apache::thrift::protocol::TType _etype991;
+            xfer += iprot->readListBegin(_etype991, _size988);
+            (*(this->success)).resize(_size988);
+            uint32_t _i992;
+            for (_i992 = 0; _i992 < _size988; ++_i992)
             {
-              xfer += iprot->readString((*(this->success))[_i986]);
+              xfer += iprot->readString((*(this->success))[_i992]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9574,14 +9574,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size987;
-            ::apache::thrift::protocol::TType _etype990;
-            xfer += iprot->readListBegin(_etype990, _size987);
-            this->new_parts.resize(_size987);
-            uint32_t _i991;
-            for (_i991 = 0; _i991 < _size987; ++_i991)
+            uint32_t _size993;
+            ::apache::thrift::protocol::TType _etype996;
+            xfer += iprot->readListBegin(_etype996, _size993);
+            this->new_parts.resize(_size993);
+            uint32_t _i997;
+            for (_i997 = 0; _i997 < _size993; ++_i997)
             {
-              xfer += this->new_parts[_i991].read(iprot);
+              xfer += this->new_parts[_i997].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9610,10 +9610,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 _iter992;
-    for (_iter992 = this->new_parts.begin(); _iter992 != this->new_parts.end(); ++_iter992)
+    std::vector<Partition> ::const_iterator _iter998;
+    for (_iter998 = this->new_parts.begin(); _iter998 != this->new_parts.end(); ++_iter998)
     {
-      xfer += (*_iter992).write(oprot);
+      xfer += (*_iter998).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9637,10 +9637,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 _iter993;
-    for (_iter993 = (*(this->new_parts)).begin(); _iter993 != (*(this->new_parts)).end(); ++_iter993)
+    std::vector<Partition> ::const_iterator _iter999;
+    for (_iter999 = (*(this->new_parts)).begin(); _iter999 != (*(this->new_parts)).end(); ++_iter999)
     {
-      xfer += (*_iter993).write(oprot);
+      xfer += (*_iter999).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9849,14 +9849,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 _size994;
-            ::apache::thrift::protocol::TType _etype997;
-            xfer += iprot->readListBegin(_etype997, _size994);
-            this->new_parts.resize(_size994);
-            uint32_t _i998;
-            for (_i998 = 0; _i998 < _size994; ++_i998)
+            uint32_t _size1000;
+            ::apache::thrift::protocol::TType _etype1003;
+            xfer += iprot->readListBegin(_etype1003, _size1000);
+            this->new_parts.resize(_size1000);
+            uint32_t _i1004;
+            for (_i1004 = 0; _i1004 < _size1000; ++_i1004)
             {
-              xfer += this->new_parts[_i998].read(iprot);
+              xfer += this->new_parts[_i1004].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9885,10 +9885,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 _iter999;
-    for (_iter999 = this->new_parts.begin(); _iter999 != this->new_parts.end(); ++_iter999)
+    std::vector<PartitionSpec> ::const_iterator _iter1005;
+    for (_iter1005 = this->new_parts.begin(); _iter1005 != this->new_parts.end(); ++_iter1005)
     {
-      xfer += (*_iter999).write(oprot);
+      xfer += (*_iter1005).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9912,10 +9912,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 _iter1000;
-    for (_iter1000 = (*(this->new_parts)).begin(); _iter1000 != (*(this->new_parts)).end(); ++_iter1000)
+    std::vector<PartitionSpec> ::const_iterator _iter1006;
+    for (_iter1006 = (*(this->new_parts)).begin(); _iter1006 != (*(this->new_parts)).end(); ++_iter1006)
     {
-      xfer += (*_iter1000).write(oprot);
+      xfer += (*_iter1006).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -10140,14 +10140,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1001;
-            ::apache::thrift::protocol::TType _etype1004;
-            xfer += iprot->readListBegin(_etype1004, _size1001);
-            this->part_vals.resize(_size1001);
-            uint32_t _i1005;
-            for (_i1005 = 0; _i1005 < _size1001; ++_i1005)
+            uint32_t _size1007;
+            ::apache::thrift::protocol::TType _etype1010;
+            xfer += iprot->readListBegin(_etype1010, _size1007);
+            this->part_vals.resize(_size1007);
+            uint32_t _i1011;
+            for (_i1011 = 0; _i1011 < _size1007; ++_i1011)
             {
-              xfer += iprot->readString(this->part_vals[_i1005]);
+              xfer += iprot->readString(this->part_vals[_i1011]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10184,10 +10184,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 _iter1006;
-    for (_iter1006 = this->part_vals.begin(); _iter1006 != this->part_vals.end(); ++_iter1006)
+    std::vector<std::string> ::const_iterator _iter1012;
+    for (_iter1012 = this->part_vals.begin(); _iter1012 != this->part_vals.end(); ++_iter1012)
     {
-      xfer += oprot->writeString((*_iter1006));
+      xfer += oprot->writeString((*_iter1012));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10219,10 +10219,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 _iter1007;
-    for (_iter1007 = (*(this->part_vals)).begin(); _iter1007 != (*(this->part_vals)).end(); ++_iter1007)
+    std::vector<std::string> ::const_iterator _iter1013;
+    for (_iter1013 = (*(this->part_vals)).begin(); _iter1013 != (*(this->part_vals)).end(); ++_iter1013)
     {
-      xfer += oprot->writeString((*_iter1007));
+      xfer += oprot->writeString((*_iter1013));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10694,14 +10694,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1008;
-            ::apache::thrift::protocol::TType _etype1011;
-            xfer += iprot->readListBegin(_etype1011, _size1008);
-            this->part_vals.resize(_size1008);
-            uint32_t _i1012;
-            for (_i1012 = 0; _i1012 < _size1008; ++_i1012)
+            uint32_t _size1014;
+            ::apache::thrift::protocol::TType _etype1017;
+            xfer += iprot->readListBegin(_etype1017, _size1014);
+            this->part_vals.resize(_size1014);
+            uint32_t _i1018;
+            for (_i1018 = 0; _i1018 < _size1014; ++_i1018)
             {
-              xfer += iprot->readString(this->part_vals[_i1012]);
+              xfer += iprot->readString(this->part_vals[_i1018]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10746,10 +10746,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 _iter1013;
-    for (_iter1013 = this->part_vals.begin(); _iter1013 != this->part_vals.end(); ++_iter1013)
+    std::vector<std::string> ::const_iterator _iter1019;
+    for (_iter1019 = this->part_vals.begin(); _iter1019 != this->part_vals.end(); ++_iter1019)
     {
-      xfer += oprot->writeString((*_iter1013));
+      xfer += oprot->writeString((*_iter1019));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10785,10 +10785,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 _iter1014;
-    for (_iter1014 = (*(this->part_vals)).begin(); _iter1014 != (*(this->part_vals)).end(); ++_iter1014)
+    std::vector<std::string> ::const_iterator _iter1020;
+    for (_iter1020 = (*(this->part_vals)).begin(); _iter1020 != (*(this->part_vals)).end(); ++_iter1020)
     {
-      xfer += oprot->writeString((*_iter1014));
+      xfer += oprot->writeString((*_iter1020));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11591,14 +11591,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         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 _size1021;
+            ::apache::thrift::protocol::TType _etype1024;
+            xfer += iprot->readListBegin(_etype1024, _size1021);
+            this->part_vals.resize(_size1021);
+            uint32_t _i1025;
+            for (_i1025 = 0; _i1025 < _size1021; ++_i1025)
             {
-              xfer += iprot->readString(this->part_vals[_i1019]);
+              xfer += iprot->readString(this->part_vals[_i1025]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11643,10 +11643,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 _iter1020;
-    for (_iter1020 = this->part_vals.begin(); _iter1020 != this->part_vals.end(); ++_iter1020)
+    std::vector<std::string> ::const_iterator _iter1026;
+    for (_iter1026 = this->part_vals.begin(); _iter1026 != this->part_vals.end(); ++_iter1026)
     {
-      xfer += oprot->writeString((*_iter1020));
+      xfer += oprot->writeString((*_iter1026));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11682,10 +11682,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 _iter1021;
-    for (_iter1021 = (*(this->part_vals)).begin(); _iter1021 != (*(this->part_vals)).end(); ++_iter1021)
+    std::vector<std::string> ::const_iterator _iter1027;
+    for (_iter1027 = (*(this->part_vals)).begin(); _iter1027 != (*(this->part_vals)).end(); ++_iter1027)
     {
-      xfer += oprot->writeString((*_iter1021));
+      xfer += oprot->writeString((*_iter1027));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11894,14 +11894,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1022;
-            ::apache::thrift::protocol::TType _etype1025;
-            xfer += iprot->readListBegin(_etype1025, _size1022);
-            this->part_vals.resize(_size1022);
-            uint32_t _i1026;
-            for (_i1026 = 0; _i1026 < _size1022; ++_i1026)
+            uint32_t _size1028;
+            ::apache::thrift::protocol::TType _etype1031;
+            xfer += iprot->readListBegin(_etype1031, _size1028);
+            this->part_vals.resize(_size1028);
+            uint32_t _i1032;
+            for (_i1032 = 0; _i1032 < _size1028; ++_i1032)
             {
-              xfer += iprot->readString(this->part_vals[_i1026]);
+              xfer += iprot->readString(this->part_vals[_i1032]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11954,10 +11954,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 _iter1027;
-    for (_iter1027 = this->part_vals.begin(); _iter1027 != this->part_vals.end(); ++_iter1027)
+    std::vector<std::string> ::const_iterator _iter1033;
+    for (_iter1033 = this->part_vals.begin(); _iter1033 != this->part_vals.end(); ++_iter1033)
     {
-      xfer += oprot->writeString((*_iter1027));
+      xfer += oprot->writeString((*_iter1033));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11997,10 +11997,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 _iter1028;
-    for (_iter1028 = (*(this->part_vals)).begin(); _iter1028 != (*(this->part_vals)).end(); ++_iter1028)
+    std::vector<std::string> ::const_iterator _iter1034;
+    for (_iter1034 = (*(this->part_vals)).begin(); _iter1034 != (*(this->part_vals)).end(); ++_iter1034)
     {
-      xfer += oprot->writeString((*_iter1028));
+      xfer += oprot->writeString((*_iter1034));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13006,14 +13006,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1029;
-            ::apache::thrift::protocol::TType _etype1032;
-            xfer += iprot->readListBegin(_etype1032, _size1029);
-            this->part_vals.resize(_size1029);
-            uint32_t _i1033;
-            for (_i1033 = 0; _i1033 < _size1029; ++_i1033)
+            uint32_t _size1035;
+            ::apache::thrift::protocol::TType _etype1038;
+            xfer += iprot->readListBegin(_etype1038, _size1035);
+            this->part_vals.resize(_size1035);
+            uint32_t _i1039;
+            for (_i1039 = 0; _i1039 < _size1035; ++_i1039)
             {
-              xfer += iprot->readString(this->part_vals[_i1033]);
+              xfer += iprot->readString(this->part_vals[_i1039]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13050,10 +13050,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 _iter1034;
-    for (_iter1034 = this->part_vals.begin(); _iter1034 != this->part_vals.end(); ++_iter1034)
+    std::vector<std::string> ::const_iterator _iter1040;
+    for (_iter1040 = this->part_vals.begin(); _iter1040 != this->part_vals.end(); ++_iter1040)
     {
-      xfer += oprot->writeString((*_iter1034));
+      xfer += oprot->writeString((*_iter1040));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13085,10 +13085,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 _iter1035;
-    for (_iter1035 = (*(this->part_vals)).begin(); _iter1035 != (*(this->part_vals)).end(); ++_iter1035)
+    std::vector<std::string> ::const_iterator _iter1041;
+    for (_iter1041 = (*(this->part_vals)).begin(); _iter1041 != (*(this->part_vals)).end(); ++_iter1041)
     {
-      xfer += oprot->writeString((*_iter1035));
+      xfer += oprot->writeString((*_iter1041));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13277,17 +13277,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1036;
-            ::apache::thrift::protocol::TType _ktype1037;
-            ::apache::thrift::protocol::TType _vtype1038;
-            xfer += iprot->readMapBegin(_ktype1037, _vtype1038, _size1036);
-            uint32_t _i1040;
-            for (_i1040 = 0; _i1040 < _size1036; ++_i1040)
+            uint32_t _size1042;
+            ::apache::thrift::protocol::TType _ktype1043;
+            ::apache::thrift::protocol::TType _vtype1044;
+            xfer += iprot->readMapBegin(_ktype1043, _vtype1044, _size1042);
+            uint32_t _i1046;
+            for (_i1046 = 0; _i1046 < _size1042; ++_i1046)
             {
-              std::string _key1041;
-              xfer += iprot->readString(_key1041);
-              std::string& _val1042 = this->partitionSpecs[_key1041];
-              xfer += iprot->readString(_val1042);
+              std::string _key1047;
+              xfer += iprot->readString(_key1047);
+              std::string& _val1048 = this->partitionSpecs[_key1047];
+              xfer += iprot->readString(_val1048);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13348,11 +13348,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 _iter1043;
-    for (_iter1043 = this->partitionSpecs.begin(); _iter1043 != this->partitionSpecs.end(); ++_iter1043)
+    std::map<std::string, std::string> ::const_iterator _iter1049;
+    for (_iter1049 = this->partitionSpecs.begin(); _iter1049 != this->partitionSpecs.end(); ++_iter1049)
     {
-      xfer += oprot->writeString(_iter1043->first);
-      xfer += oprot->writeString(_iter1043->second);
+      xfer += oprot->writeString(_iter1049->first);
+      xfer += oprot->writeString(_iter1049->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13392,11 +13392,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 _iter1044;
-    for (_iter1044 = (*(this->partitionSpecs)).begin(); _iter1044 != (*(this->partitionSpecs)).end(); ++_iter1044)
+    std::map<std::string, std::string> ::const_iterator _iter1050;
+    for (_iter1050 = (*(this->partitionSpecs)).begin(); _iter1050 != (*(this->partitionSpecs)).end(); ++_iter1050)
     {
-      xfer += oprot->writeString(_iter1044->first);
-      xfer += oprot->writeString(_iter1044->second);
+      xfer += oprot->writeString(_iter1050->first);
+      xfer += oprot->writeString(_iter1050->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13641,17 +13641,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size1045;
-            ::apache::thrift::protocol::TType _ktype1046;
-            ::apache::thrift::protocol::TType _vtype1047;
-            xfer += iprot->readMapBegin(_ktype1046, _vtype1047, _size1045);
-            uint32_t _i1049;
-            for (_i1049 = 0; _i1049 < _size1045; ++_i1049)
+            uint32_t _size1051;
+            ::apache::thrift::protocol::TType _ktype1052;
+            ::apache::thrift::protocol::TType _vtype1053;
+            xfer += iprot->readMapBegin(_ktype1052, _vtype1053, _size1051);
+            uint32_t _i1055;
+            for (_i1055 = 0; _i1055 < _size1051; ++_i1055)
             {
-              std::string _key1050;
-              xfer += iprot->readString(_key1050);
-              std::string& _val1051 = this->partitionSpecs[_key1050];
-              xfer += iprot->readString(_val1051);
+              std::string _key1056;
+              xfer += iprot->readString(_key1056);
+              std::string& _val1057 = this->partitionSpecs[_key1056];
+              xfer += iprot->readString(_val1057);
             }
             xfer += iprot->readMapEnd();
           }
@@ -13712,11 +13712,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 _iter1052;
-    for (_iter1052 = this->partitionSpecs.begin(); _iter1052 != this->partitionSpecs.end(); ++_iter1052)
+    std::map<std::string, std::string> ::const_iterator _iter1058;
+    for (_iter1058 = this->partitionSpecs.begin(); _iter1058 != this->partitionSpecs.end(); ++_iter1058)
     {
-      xfer += oprot->writeString(_iter1052->first);
-      xfer += oprot->writeString(_iter1052->second);
+      xfer += oprot->writeString(_iter1058->first);
+      xfer += oprot->writeString(_iter1058->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13756,11 +13756,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 _iter1053;
-    for (_iter1053 = (*(this->partitionSpecs)).begin(); _iter1053 != (*(this->partitionSpecs)).end(); ++_iter1053)
+    std::map<std::string, std::string> ::const_iterator _iter1059;
+    for (_iter1059 = (*(this->partitionSpecs)).begin(); _iter1059 != (*(this->partitionSpecs)).end(); ++_iter1059)
     {
-      xfer += oprot->writeString(_iter1053->first);
-      xfer += oprot->writeString(_iter1053->second);
+      xfer += oprot->writeString(_iter1059->first);
+      xfer += oprot->writeString(_iter1059->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -13817,14 +13817,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1054;
-            ::apache::thrift::protocol::TType _etype1057;
-            xfer += iprot->readListBegin(_etype1057, _size1054);
-            this->success.resize(_size1054);
-            uint32_t _i1058;
-            for (_i1058 = 0; _i1058 < _size1054; ++_i1058)
+            uint32_t _size1060;
+            ::apache::thrift::protocol::TType _etype1063;
+            xfer += iprot->readListBegin(_etype1063, _size1060);
+            this->success.resize(_size1060);
+            uint32_t _i1064;
+            for (_i1064 = 0; _i1064 < _size1060; ++_i1064)
             {
-              xfer += this->success[_i1058].read(iprot);
+              xfer += this->success[_i1064].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13887,10 +13887,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 _iter1059;
-      for (_iter1059 = this->success.begin(); _iter1059 != this->success.end(); ++_iter1059)
+      std::vector<Partition> ::const_iterator _iter1065;
+      for (_iter1065 = this->success.begin(); _iter1065 != this->success.end(); ++_iter1065)
       {
-        xfer += (*_iter1059).write(oprot);
+        xfer += (*_iter1065).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13947,14 +13947,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1060;
-            ::apache::thrift::protocol::TType _etype1063;
-            xfer += iprot->readListBegin(_etype1063, _size1060);
-            (*(this->success)).resize(_size1060);
-            uint32_t _i1064;
-            for (_i1064 = 0; _i1064 < _size1060; ++_i1064)
+            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))[_i1064].read(iprot);
+              xfer += (*(this->success))[_i1070].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14053,14 +14053,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 _size1065;
-            ::apache::thrift::protocol::TType _etype1068;
-            xfer += iprot->readListBegin(_etype1068, _size1065);
-            this->part_vals.resize(_size1065);
-            uint32_t _i1069;
-            for (_i1069 = 0; _i1069 < _size1065; ++_i1069)
+            uint32_t _size1071;
+            ::apache::thrift::protocol::TType _etype1074;
+            xfer += iprot->readListBegin(_etype1074, _size1071);
+            this->part_vals.resize(_size1071);
+            uint32_t _i1075;
+            for (_i1075 = 0; _i1075 < _size1071; ++_i1075)
             {
-              xfer += iprot->readString(this->part_vals[_i1069]);
+              xfer += iprot->readString(this->part_vals[_i1075]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14081,14 +14081,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 _size1070;
-            ::apache::thrift::protocol::TType _etype1073;
-            xfer += iprot->readListBegin(_etype1073, _size1070);
-            this->group_names.resize(_size1070);
-            uint32_t _i1074;
-            for (_i1074 = 0; _i1074 < _size1070; ++_i1074)
+            uint32_t _size1076;
+            ::apache::thrift::protocol::TType _etype1079;
+            xfer += iprot->readListBegin(_etype1079, _size1076);
+            this->group_names.resize(_size1076);
+            uint32_t _i1080;
+            for (_i1080 = 0; _i1080 < _size1076; ++_i1080)
             {
-              xfer += iprot->readString(this->group_names[_i1074]);
+              xfer += iprot->readString(this->group_names[_i1080]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14125,10 +14125,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 _iter1075;
-    for (_iter1075 = this->part_vals.begin(); _iter1075 != this->part_vals.end(); ++_iter1075)
+    std::vector<std::string> ::const_iterator _iter1081;
+    for (_iter1081 = this->part_vals.begin(); _iter1081 != this->part_vals.end(); ++_iter1081)
     {
-      xfer += oprot->writeString((*_iter1075));
+      xfer += oprot->writeString((*_iter1081));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14141,10 +14141,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 _iter1076;
-    for (_iter1076 = this->group_names.begin(); _iter1076 != this->group_names.end(); ++_iter1076)
+    std::vector<std::string> ::const_iterator _iter1082;
+    for (_iter1082 = this->group_names.begin(); _iter1082 != this->group_names.end(); ++_iter1082)
     {
-      xfer += oprot->writeString((*_iter1076));
+      xfer += oprot->writeString((*_iter1082));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14176,10 +14176,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 _iter1077;
-    for (_iter1077 = (*(this->part_vals)).begin(); _iter1077 != (*(this->part_vals)).end(); ++_iter1077)
+    std::vector<std::string> ::const_iterator _iter1083;
+    for (_iter1083 = (*(this->part_vals)).begin(); _iter1083 != (*(this->part_vals)).end(); ++_iter1083)
     {
-      xfer += oprot->writeString((*_iter1077));
+      xfer += oprot->writeString((*_iter1083));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14192,10 +14192,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 _iter1078;
-    for (_iter1078 = (*(this->group_names)).begin(); _iter1078 != (*(this->group_names)).end(); ++_iter1078)
+    std::vector<std::string> ::const_iterator _iter1084;
+    for (_iter1084 = (*(this->group_names)).begin(); _iter1084 != (*(this->group_names)).end(); ++_iter1084)
     {
-      xfer += oprot->writeString((*_iter1078));
+      xfer += oprot->writeString((*_iter1084));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14754,14 +14754,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1079;
-            ::apache::thrift::protocol::TType _etype1082;
-            xfer += iprot->readListBegin(_etype1082, _size1079);
-            this->success.resize(_size1079);
-            uint32_t _i1083;
-            for (_i1083 = 0; _i1083 < _size1079; ++_i1083)
+            uint32_t _size1085;
+            ::apache::thrift::protocol::TType _etype1088;
+            xfer += iprot->readListBegin(_etype1088, _size1085);
+            this->success.resize(_size1085);
+            uint32_t _i1089;
+            for (_i1089 = 0; _i1089 < _size1085; ++_i1089)
             {
-              xfer += this->success[_i1083].read(iprot);
+              xfer += this->success[_i1089].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14808,10 +14808,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 _iter1084;
-      for (_iter1084 = this->success.begin(); _iter1084 != this->success.end(); ++_iter1084)
+      std::vector<Partition> ::const_iterator _iter1090;
+      for (_iter1090 = this->success.begin(); _iter1090 != this->success.end(); ++_iter1090)
       {
-        xfer += (*_iter1084).write(oprot);
+        xfer += (*_iter1090).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14860,14 +14860,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1085;
-            ::apache::thrift::protocol::TType _etype1088;
-            xfer += iprot->readListBegin(_etype1088, _size1085);
-            (*(this->success)).resize(_size1085);
-            uint32_t _i1089;
-            for (_i1089 = 0; _i1089 < _size1085; ++_i1089)
+            uint32_t _size1091;
+            ::apache::thrift::protocol::TType _etype1094;
+            xfer += iprot->readListBegin(_etype1094, _size1091);
+            (*(this->success)).resize(_size1091);
+            uint32_t _i1095;
+            for (_i1095 = 0; _i1095 < _size1091; ++_i1095)
             {
-              xfer += (*(this->success))[_i1089].read(iprot);
+              xfer += (*(this->success))[_i1095].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14966,14 +14966,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 _size1090;
-            ::apache::thrift::protocol::TType _etype1093;
-            xfer += iprot->readListBegin(_etype1093, _size1090);
-            this->group_names.resize(_size1090);
-            uint32_t _i1094;
-            for (_i1094 = 0; _i1094 < _size1090; ++_i1094)
+            uint32_t _size1096;
+            ::apache::thrift::protocol::TType _etype1099;
+            xfer += iprot->readListBegin(_etype1099, _size1096);
+            this->group_names.resize(_size1096);
+            uint32_t _i1100;
+            for (_i1100 = 0; _i1100 < _size1096; ++_i1100)
             {
-              xfer += iprot->readString(this->group_names[_i1094]);
+              xfer += iprot->readString(this->group_names[_i1100]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15018,10 +15018,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 _iter1095;
-    for (_iter1095 = this->group_names.begin(); _iter1095 != this->group_names.end(); ++_iter1095)
+    std::vector<std::string> ::const_iterator _iter1101;
+    for (_iter1101 = this->group_names.begin(); _iter1101 != this->group_names.end(); ++_iter1101)
     {
-      xfer += oprot->writeString((*_iter1095));
+      xfer += oprot->writeString((*_iter1101));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15061,10 +15061,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 _iter1096;
-    for (_iter1096 = (*(this->group_names)).begin(); _iter1096 != (*(this->group_names)).end(); ++_iter1096)
+    std::vector<std::string> ::const_iterator _iter1102;
+    for (_iter1102 = (*(this->group_names)).begin(); _iter1102 != (*(this->group_names)).end(); ++_iter1102)
     {
-      xfer += oprot->writeString((*_iter1096));
+      xfer += oprot->writeString((*_iter1102));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15105,14 +15105,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1097;
-            ::apache::thrift::protocol::TType _etype1100;
-            xfer += iprot->readListBegin(_etype1100, _size1097);
-            this->success.resize(_size1097);
-            uint32_t _i1101;
-            for (_i1101 = 0; _i1101 < _size1097; ++_i1101)
+            uint32_t _size1103;
+            ::apache::thrift::protocol::TType _etype1106;
+            xfer += iprot->readListBegin(_etype1106, _size1103);
+            this->success.resize(_size1103);
+            uint32_t _i1107;
+            for (_i1107 = 0; _i1107 < _size1103; ++_i1107)
             {
-              xfer += this->success[_i1101].read(iprot);
+              xfer += this->success[_i1107].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15159,10 +15159,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 _iter1102;
-      for (_iter1102 = this->success.begin(); _iter1102 != this->success.end(); ++_iter1102)
+      std::vector<Partition> ::const_iterator _iter1108;
+      for (_iter1108 = this->success.begin(); _iter1108 != this->success.end(); ++_iter1108)
       {
-        xfer += (*_iter1102).write(oprot);
+        xfer += (*_iter1108).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15211,14 +15211,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1103;
-            ::apache::thrift::protocol::TType _etype1106;
-            xfer += iprot->readListBegin(_etype1106, _size1103);
-            (*(this->success)).resize(_size1103);
-            uint32_t _i1107;
-            for (_i1107 = 0; _i1107 < _size1103; ++_i1107)
+            uint32_t _size1109;
+            ::apache::thrift::protocol::TType _etype1112;
+            xfer += iprot->readListBegin(_etype1112, _size1109);
+            (*(this->success)).resize(_size1109);
+            uint32_t _i1113;
+            for (_i1113 = 0; _i1113 < _size1109; ++_i1113)
             {
-              xfer += (*(this->success))[_i1107].read(iprot);
+              xfer += (*(this->success))[_i1113].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15396,14 +15396,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1108;
-            ::apache::thrift::protocol::TType _etype1111;
-            xfer += iprot->readListBegin(_etype1111, _size1108);
-            this->success.resize(_size1108);
-            uint32_t _i1112;
-            for (_i1112 = 0; _i1112 < _size1108; ++_i1112)
+            uint32_t _size1114;
+            ::apache::thrift::protocol::TType _etype1117;
+            xfer += iprot->readListBegin(_etype1117, _siz

<TRUNCATED>

[5/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 5cf880a..7aebede 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 _list602 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list602.size);
-                long _elem603;
-                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
+                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)
                 {
-                  _elem603 = iprot.readI64();
-                  struct.fileIds.add(_elem603);
+                  _elem611 = iprot.readI64();
+                  struct.fileIds.add(_elem611);
                 }
                 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 _iter605 : struct.fileIds)
+          for (long _iter613 : struct.fileIds)
           {
-            oprot.writeI64(_iter605);
+            oprot.writeI64(_iter613);
           }
           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 _iter606 : struct.fileIds)
+        for (long _iter614 : struct.fileIds)
         {
-          oprot.writeI64(_iter606);
+          oprot.writeI64(_iter614);
         }
       }
     }
@@ -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 _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)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list615.size);
+        long _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem608 = iprot.readI64();
-          struct.fileIds.add(_elem608);
+          _elem616 = iprot.readI64();
+          struct.fileIds.add(_elem616);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 8870c2f..fe83a6e 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 _map592 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map592.size);
-                long _key593;
-                ByteBuffer _val594;
-                for (int _i595 = 0; _i595 < _map592.size; ++_i595)
+                org.apache.thrift.protocol.TMap _map600 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map600.size);
+                long _key601;
+                ByteBuffer _val602;
+                for (int _i603 = 0; _i603 < _map600.size; ++_i603)
                 {
-                  _key593 = iprot.readI64();
-                  _val594 = iprot.readBinary();
-                  struct.metadata.put(_key593, _val594);
+                  _key601 = iprot.readI64();
+                  _val602 = iprot.readBinary();
+                  struct.metadata.put(_key601, _val602);
                 }
                 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> _iter596 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter604 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter596.getKey());
-            oprot.writeBinary(_iter596.getValue());
+            oprot.writeI64(_iter604.getKey());
+            oprot.writeBinary(_iter604.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> _iter597 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter605 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter597.getKey());
-          oprot.writeBinary(_iter597.getValue());
+          oprot.writeI64(_iter605.getKey());
+          oprot.writeBinary(_iter605.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 _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)
+        org.apache.thrift.protocol.TMap _map606 = 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*_map606.size);
+        long _key607;
+        ByteBuffer _val608;
+        for (int _i609 = 0; _i609 < _map606.size; ++_i609)
         {
-          _key599 = iprot.readI64();
-          _val600 = iprot.readBinary();
-          struct.metadata.put(_key599, _val600);
+          _key607 = iprot.readI64();
+          _val608 = iprot.readBinary();
+          struct.metadata.put(_key607, _val608);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index de4b2f6..225fda9 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -525,13 +525,13 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list650.size);
-                String _elem651;
-                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list658.size);
+                String _elem659;
+                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                 {
-                  _elem651 = iprot.readString();
-                  struct.tblNames.add(_elem651);
+                  _elem659 = iprot.readString();
+                  struct.tblNames.add(_elem659);
                 }
                 iprot.readListEnd();
               }
@@ -572,9 +572,9 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter653 : struct.tblNames)
+            for (String _iter661 : struct.tblNames)
             {
-              oprot.writeString(_iter653);
+              oprot.writeString(_iter661);
             }
             oprot.writeListEnd();
           }
@@ -617,9 +617,9 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter654 : struct.tblNames)
+          for (String _iter662 : struct.tblNames)
           {
-            oprot.writeString(_iter654);
+            oprot.writeString(_iter662);
           }
         }
       }
@@ -636,13 +636,13 @@ public class GetTablesRequest implements org.apache.thrift.TBase<GetTablesReques
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list655.size);
-          String _elem656;
-          for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+          org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list663.size);
+          String _elem664;
+          for (int _i665 = 0; _i665 < _list663.size; ++_i665)
           {
-            _elem656 = iprot.readString();
-            struct.tblNames.add(_elem656);
+            _elem664 = iprot.readString();
+            struct.tblNames.add(_elem664);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 4c456f2..91cb198 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list658.size);
-                Table _elem659;
-                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list666.size);
+                Table _elem667;
+                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                 {
-                  _elem659 = new Table();
-                  _elem659.read(iprot);
-                  struct.tables.add(_elem659);
+                  _elem667 = new Table();
+                  _elem667.read(iprot);
+                  struct.tables.add(_elem667);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter661 : struct.tables)
+          for (Table _iter669 : struct.tables)
           {
-            _iter661.write(oprot);
+            _iter669.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter662 : struct.tables)
+        for (Table _iter670 : struct.tables)
         {
-          _iter662.write(oprot);
+          _iter670.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ public class GetTablesResult implements org.apache.thrift.TBase<GetTablesResult,
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list663.size);
-        Table _elem664;
-        for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+        org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list671.size);
+        Table _elem672;
+        for (int _i673 = 0; _i673 < _list671.size; ++_i673)
         {
-          _elem664 = new Table();
-          _elem664.read(iprot);
-          struct.tables.add(_elem664);
+          _elem672 = new Table();
+          _elem672.read(iprot);
+          struct.tables.add(_elem672);
         }
       }
       struct.setTablesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 a8df524..39a607d 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
@@ -39,6 +39,7 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InsertEventRequestData");
 
   private static final org.apache.thrift.protocol.TField FILES_ADDED_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAdded", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField FILES_ADDED_CHECKSUM_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAddedChecksum", org.apache.thrift.protocol.TType.LIST, (short)2);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,10 +48,12 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
   }
 
   private List<String> filesAdded; // required
+  private List<ByteBuffer> filesAddedChecksum; // 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 {
-    FILES_ADDED((short)1, "filesAdded");
+    FILES_ADDED((short)1, "filesAdded"),
+    FILES_ADDED_CHECKSUM((short)2, "filesAddedChecksum");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -67,6 +70,8 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       switch(fieldId) {
         case 1: // FILES_ADDED
           return FILES_ADDED;
+        case 2: // FILES_ADDED_CHECKSUM
+          return FILES_ADDED_CHECKSUM;
         default:
           return null;
       }
@@ -107,12 +112,16 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.FILES_ADDED_CHECKSUM};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.FILES_ADDED, new org.apache.thrift.meta_data.FieldMetaData("filesAdded", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.FILES_ADDED_CHECKSUM, new org.apache.thrift.meta_data.FieldMetaData("filesAddedChecksum", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(InsertEventRequestData.class, metaDataMap);
   }
@@ -135,6 +144,10 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       List<String> __this__filesAdded = new ArrayList<String>(other.filesAdded);
       this.filesAdded = __this__filesAdded;
     }
+    if (other.isSetFilesAddedChecksum()) {
+      List<ByteBuffer> __this__filesAddedChecksum = new ArrayList<ByteBuffer>(other.filesAddedChecksum);
+      this.filesAddedChecksum = __this__filesAddedChecksum;
+    }
   }
 
   public InsertEventRequestData deepCopy() {
@@ -144,6 +157,7 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
   @Override
   public void clear() {
     this.filesAdded = null;
+    this.filesAddedChecksum = null;
   }
 
   public int getFilesAddedSize() {
@@ -184,6 +198,44 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     }
   }
 
+  public int getFilesAddedChecksumSize() {
+    return (this.filesAddedChecksum == null) ? 0 : this.filesAddedChecksum.size();
+  }
+
+  public java.util.Iterator<ByteBuffer> getFilesAddedChecksumIterator() {
+    return (this.filesAddedChecksum == null) ? null : this.filesAddedChecksum.iterator();
+  }
+
+  public void addToFilesAddedChecksum(ByteBuffer elem) {
+    if (this.filesAddedChecksum == null) {
+      this.filesAddedChecksum = new ArrayList<ByteBuffer>();
+    }
+    this.filesAddedChecksum.add(elem);
+  }
+
+  public List<ByteBuffer> getFilesAddedChecksum() {
+    return this.filesAddedChecksum;
+  }
+
+  public void setFilesAddedChecksum(List<ByteBuffer> filesAddedChecksum) {
+    this.filesAddedChecksum = filesAddedChecksum;
+  }
+
+  public void unsetFilesAddedChecksum() {
+    this.filesAddedChecksum = null;
+  }
+
+  /** Returns true if field filesAddedChecksum is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilesAddedChecksum() {
+    return this.filesAddedChecksum != null;
+  }
+
+  public void setFilesAddedChecksumIsSet(boolean value) {
+    if (!value) {
+      this.filesAddedChecksum = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case FILES_ADDED:
@@ -194,6 +246,14 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       }
       break;
 
+    case FILES_ADDED_CHECKSUM:
+      if (value == null) {
+        unsetFilesAddedChecksum();
+      } else {
+        setFilesAddedChecksum((List<ByteBuffer>)value);
+      }
+      break;
+
     }
   }
 
@@ -202,6 +262,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     case FILES_ADDED:
       return getFilesAdded();
 
+    case FILES_ADDED_CHECKSUM:
+      return getFilesAddedChecksum();
+
     }
     throw new IllegalStateException();
   }
@@ -215,6 +278,8 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     switch (field) {
     case FILES_ADDED:
       return isSetFilesAdded();
+    case FILES_ADDED_CHECKSUM:
+      return isSetFilesAddedChecksum();
     }
     throw new IllegalStateException();
   }
@@ -241,6 +306,15 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         return false;
     }
 
+    boolean this_present_filesAddedChecksum = true && this.isSetFilesAddedChecksum();
+    boolean that_present_filesAddedChecksum = true && that.isSetFilesAddedChecksum();
+    if (this_present_filesAddedChecksum || that_present_filesAddedChecksum) {
+      if (!(this_present_filesAddedChecksum && that_present_filesAddedChecksum))
+        return false;
+      if (!this.filesAddedChecksum.equals(that.filesAddedChecksum))
+        return false;
+    }
+
     return true;
   }
 
@@ -253,6 +327,11 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     if (present_filesAdded)
       list.add(filesAdded);
 
+    boolean present_filesAddedChecksum = true && (isSetFilesAddedChecksum());
+    list.add(present_filesAddedChecksum);
+    if (present_filesAddedChecksum)
+      list.add(filesAddedChecksum);
+
     return list.hashCode();
   }
 
@@ -274,6 +353,16 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetFilesAddedChecksum()).compareTo(other.isSetFilesAddedChecksum());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilesAddedChecksum()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filesAddedChecksum, other.filesAddedChecksum);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -301,6 +390,16 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       sb.append(this.filesAdded);
     }
     first = false;
+    if (isSetFilesAddedChecksum()) {
+      if (!first) sb.append(", ");
+      sb.append("filesAddedChecksum:");
+      if (this.filesAddedChecksum == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.filesAddedChecksum, sb);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -366,6 +465,24 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 2: // FILES_ADDED_CHECKSUM
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list561 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<ByteBuffer>(_list561.size);
+                ByteBuffer _elem562;
+                for (int _i563 = 0; _i563 < _list561.size; ++_i563)
+                {
+                  _elem562 = iprot.readBinary();
+                  struct.filesAddedChecksum.add(_elem562);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFilesAddedChecksumIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -383,14 +500,28 @@ 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 _iter561 : struct.filesAdded)
+          for (String _iter564 : struct.filesAdded)
           {
-            oprot.writeString(_iter561);
+            oprot.writeString(_iter564);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.filesAddedChecksum != null) {
+        if (struct.isSetFilesAddedChecksum()) {
+          oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
+            for (ByteBuffer _iter565 : struct.filesAddedChecksum)
+            {
+              oprot.writeBinary(_iter565);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -410,9 +541,23 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter562 : struct.filesAdded)
+        for (String _iter566 : struct.filesAdded)
         {
-          oprot.writeString(_iter562);
+          oprot.writeString(_iter566);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.isSetFilesAddedChecksum()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetFilesAddedChecksum()) {
+        {
+          oprot.writeI32(struct.filesAddedChecksum.size());
+          for (ByteBuffer _iter567 : struct.filesAddedChecksum)
+          {
+            oprot.writeBinary(_iter567);
+          }
         }
       }
     }
@@ -421,16 +566,30 @@ 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 _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)
+        org.apache.thrift.protocol.TList _list568 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list568.size);
+        String _elem569;
+        for (int _i570 = 0; _i570 < _list568.size; ++_i570)
         {
-          _elem564 = iprot.readString();
-          struct.filesAdded.add(_elem564);
+          _elem569 = iprot.readString();
+          struct.filesAdded.add(_elem569);
         }
       }
       struct.setFilesAddedIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<ByteBuffer>(_list571.size);
+          ByteBuffer _elem572;
+          for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+          {
+            _elem572 = iprot.readBinary();
+            struct.filesAddedChecksum.add(_elem572);
+          }
+        }
+        struct.setFilesAddedChecksumIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 6eff25d..0a1302f 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 _list610 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list610.size);
-                long _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list618.size);
+                long _elem619;
+                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
                 {
-                  _elem611 = iprot.readI64();
-                  struct.fileIds.add(_elem611);
+                  _elem619 = iprot.readI64();
+                  struct.fileIds.add(_elem619);
                 }
                 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 _list613 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list613.size);
-                ByteBuffer _elem614;
-                for (int _i615 = 0; _i615 < _list613.size; ++_i615)
+                org.apache.thrift.protocol.TList _list621 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list621.size);
+                ByteBuffer _elem622;
+                for (int _i623 = 0; _i623 < _list621.size; ++_i623)
                 {
-                  _elem614 = iprot.readBinary();
-                  struct.metadata.add(_elem614);
+                  _elem622 = iprot.readBinary();
+                  struct.metadata.add(_elem622);
                 }
                 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 _iter616 : struct.fileIds)
+          for (long _iter624 : struct.fileIds)
           {
-            oprot.writeI64(_iter616);
+            oprot.writeI64(_iter624);
           }
           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 _iter617 : struct.metadata)
+          for (ByteBuffer _iter625 : struct.metadata)
           {
-            oprot.writeBinary(_iter617);
+            oprot.writeBinary(_iter625);
           }
           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 _iter618 : struct.fileIds)
+        for (long _iter626 : struct.fileIds)
         {
-          oprot.writeI64(_iter618);
+          oprot.writeI64(_iter626);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter619 : struct.metadata)
+        for (ByteBuffer _iter627 : struct.metadata)
         {
-          oprot.writeBinary(_iter619);
+          oprot.writeBinary(_iter627);
         }
       }
       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 _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)
+        org.apache.thrift.protocol.TList _list628 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list628.size);
+        long _elem629;
+        for (int _i630 = 0; _i630 < _list628.size; ++_i630)
         {
-          _elem621 = iprot.readI64();
-          struct.fileIds.add(_elem621);
+          _elem629 = iprot.readI64();
+          struct.fileIds.add(_elem629);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        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)
+        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list631.size);
+        ByteBuffer _elem632;
+        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
         {
-          _elem624 = iprot.readBinary();
-          struct.metadata.add(_elem624);
+          _elem632 = iprot.readBinary();
+          struct.metadata.add(_elem632);
         }
       }
       struct.setMetadataIsSet(true);


[6/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 a74e28b..1311b20 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -16108,6 +16108,11 @@ void InsertEventRequestData::__set_filesAdded(const std::vector<std::string> & v
   this->filesAdded = val;
 }
 
+void InsertEventRequestData::__set_filesAddedChecksum(const std::vector<std::string> & val) {
+  this->filesAddedChecksum = val;
+__isset.filesAddedChecksum = true;
+}
+
 uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -16150,6 +16155,26 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
           xfer += iprot->skip(ftype);
         }
         break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->filesAddedChecksum.clear();
+            uint32_t _size656;
+            ::apache::thrift::protocol::TType _etype659;
+            xfer += iprot->readListBegin(_etype659, _size656);
+            this->filesAddedChecksum.resize(_size656);
+            uint32_t _i660;
+            for (_i660 = 0; _i660 < _size656; ++_i660)
+            {
+              xfer += iprot->readBinary(this->filesAddedChecksum[_i660]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.filesAddedChecksum = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -16172,15 +16197,28 @@ 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 _iter656;
-    for (_iter656 = this->filesAdded.begin(); _iter656 != this->filesAdded.end(); ++_iter656)
+    std::vector<std::string> ::const_iterator _iter661;
+    for (_iter661 = this->filesAdded.begin(); _iter661 != this->filesAdded.end(); ++_iter661)
     {
-      xfer += oprot->writeString((*_iter656));
+      xfer += oprot->writeString((*_iter661));
     }
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.filesAddedChecksum) {
+    xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 2);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAddedChecksum.size()));
+      std::vector<std::string> ::const_iterator _iter662;
+      for (_iter662 = this->filesAddedChecksum.begin(); _iter662 != this->filesAddedChecksum.end(); ++_iter662)
+      {
+        xfer += oprot->writeBinary((*_iter662));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -16189,19 +16227,26 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
 void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   using ::std::swap;
   swap(a.filesAdded, b.filesAdded);
+  swap(a.filesAddedChecksum, b.filesAddedChecksum);
+  swap(a.__isset, b.__isset);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other657) {
-  filesAdded = other657.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other663) {
+  filesAdded = other663.filesAdded;
+  filesAddedChecksum = other663.filesAddedChecksum;
+  __isset = other663.__isset;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other658) {
-  filesAdded = other658.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other664) {
+  filesAdded = other664.filesAdded;
+  filesAddedChecksum = other664.filesAddedChecksum;
+  __isset = other664.__isset;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
   using ::apache::thrift::to_string;
   out << "InsertEventRequestData(";
   out << "filesAdded=" << to_string(filesAdded);
+  out << ", " << "filesAddedChecksum="; (__isset.filesAddedChecksum ? (out << to_string(filesAddedChecksum)) : (out << "<null>"));
   out << ")";
 }
 
@@ -16275,13 +16320,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other659) {
-  insertData = other659.insertData;
-  __isset = other659.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other665) {
+  insertData = other665.insertData;
+  __isset = other665.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other660) {
-  insertData = other660.insertData;
-  __isset = other660.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other666) {
+  insertData = other666.insertData;
+  __isset = other666.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -16378,14 +16423,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size661;
-            ::apache::thrift::protocol::TType _etype664;
-            xfer += iprot->readListBegin(_etype664, _size661);
-            this->partitionVals.resize(_size661);
-            uint32_t _i665;
-            for (_i665 = 0; _i665 < _size661; ++_i665)
+            uint32_t _size667;
+            ::apache::thrift::protocol::TType _etype670;
+            xfer += iprot->readListBegin(_etype670, _size667);
+            this->partitionVals.resize(_size667);
+            uint32_t _i671;
+            for (_i671 = 0; _i671 < _size667; ++_i671)
             {
-              xfer += iprot->readString(this->partitionVals[_i665]);
+              xfer += iprot->readString(this->partitionVals[_i671]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16437,10 +16482,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 _iter666;
-      for (_iter666 = this->partitionVals.begin(); _iter666 != this->partitionVals.end(); ++_iter666)
+      std::vector<std::string> ::const_iterator _iter672;
+      for (_iter672 = this->partitionVals.begin(); _iter672 != this->partitionVals.end(); ++_iter672)
       {
-        xfer += oprot->writeString((*_iter666));
+        xfer += oprot->writeString((*_iter672));
       }
       xfer += oprot->writeListEnd();
     }
@@ -16461,21 +16506,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other667) {
-  successful = other667.successful;
-  data = other667.data;
-  dbName = other667.dbName;
-  tableName = other667.tableName;
-  partitionVals = other667.partitionVals;
-  __isset = other667.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other668) {
-  successful = other668.successful;
-  data = other668.data;
-  dbName = other668.dbName;
-  tableName = other668.tableName;
-  partitionVals = other668.partitionVals;
-  __isset = other668.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other673) {
+  successful = other673.successful;
+  data = other673.data;
+  dbName = other673.dbName;
+  tableName = other673.tableName;
+  partitionVals = other673.partitionVals;
+  __isset = other673.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other674) {
+  successful = other674.successful;
+  data = other674.data;
+  dbName = other674.dbName;
+  tableName = other674.tableName;
+  partitionVals = other674.partitionVals;
+  __isset = other674.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -16538,11 +16583,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other669) {
-  (void) other669;
+FireEventResponse::FireEventResponse(const FireEventResponse& other675) {
+  (void) other675;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other670) {
-  (void) other670;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other676) {
+  (void) other676;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -16642,15 +16687,15 @@ void swap(MetadataPpdResult &a, MetadataPpdResult &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other671) {
-  metadata = other671.metadata;
-  includeBitset = other671.includeBitset;
-  __isset = other671.__isset;
+MetadataPpdResult::MetadataPpdResult(const MetadataPpdResult& other677) {
+  metadata = other677.metadata;
+  includeBitset = other677.includeBitset;
+  __isset = other677.__isset;
 }
-MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other672) {
-  metadata = other672.metadata;
-  includeBitset = other672.includeBitset;
-  __isset = other672.__isset;
+MetadataPpdResult& MetadataPpdResult::operator=(const MetadataPpdResult& other678) {
+  metadata = other678.metadata;
+  includeBitset = other678.includeBitset;
+  __isset = other678.__isset;
   return *this;
 }
 void MetadataPpdResult::printTo(std::ostream& out) const {
@@ -16701,17 +16746,17 @@ uint32_t GetFileMetadataByExprResult::read(::apache::thrift::protocol::TProtocol
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size673;
-            ::apache::thrift::protocol::TType _ktype674;
-            ::apache::thrift::protocol::TType _vtype675;
-            xfer += iprot->readMapBegin(_ktype674, _vtype675, _size673);
-            uint32_t _i677;
-            for (_i677 = 0; _i677 < _size673; ++_i677)
+            uint32_t _size679;
+            ::apache::thrift::protocol::TType _ktype680;
+            ::apache::thrift::protocol::TType _vtype681;
+            xfer += iprot->readMapBegin(_ktype680, _vtype681, _size679);
+            uint32_t _i683;
+            for (_i683 = 0; _i683 < _size679; ++_i683)
             {
-              int64_t _key678;
-              xfer += iprot->readI64(_key678);
-              MetadataPpdResult& _val679 = this->metadata[_key678];
-              xfer += _val679.read(iprot);
+              int64_t _key684;
+              xfer += iprot->readI64(_key684);
+              MetadataPpdResult& _val685 = this->metadata[_key684];
+              xfer += _val685.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -16752,11 +16797,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 _iter680;
-    for (_iter680 = this->metadata.begin(); _iter680 != this->metadata.end(); ++_iter680)
+    std::map<int64_t, MetadataPpdResult> ::const_iterator _iter686;
+    for (_iter686 = this->metadata.begin(); _iter686 != this->metadata.end(); ++_iter686)
     {
-      xfer += oprot->writeI64(_iter680->first);
-      xfer += _iter680->second.write(oprot);
+      xfer += oprot->writeI64(_iter686->first);
+      xfer += _iter686->second.write(oprot);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -16777,13 +16822,13 @@ void swap(GetFileMetadataByExprResult &a, GetFileMetadataByExprResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other681) {
-  metadata = other681.metadata;
-  isSupported = other681.isSupported;
+GetFileMetadataByExprResult::GetFileMetadataByExprResult(const GetFileMetadataByExprResult& other687) {
+  metadata = other687.metadata;
+  isSupported = other687.isSupported;
 }
-GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other682) {
-  metadata = other682.metadata;
-  isSupported = other682.isSupported;
+GetFileMetadataByExprResult& GetFileMetadataByExprResult::operator=(const GetFileMetadataByExprResult& other688) {
+  metadata = other688.metadata;
+  isSupported = other688.isSupported;
   return *this;
 }
 void GetFileMetadataByExprResult::printTo(std::ostream& out) const {
@@ -16844,14 +16889,14 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size683;
-            ::apache::thrift::protocol::TType _etype686;
-            xfer += iprot->readListBegin(_etype686, _size683);
-            this->fileIds.resize(_size683);
-            uint32_t _i687;
-            for (_i687 = 0; _i687 < _size683; ++_i687)
+            uint32_t _size689;
+            ::apache::thrift::protocol::TType _etype692;
+            xfer += iprot->readListBegin(_etype692, _size689);
+            this->fileIds.resize(_size689);
+            uint32_t _i693;
+            for (_i693 = 0; _i693 < _size689; ++_i693)
             {
-              xfer += iprot->readI64(this->fileIds[_i687]);
+              xfer += iprot->readI64(this->fileIds[_i693]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16878,9 +16923,9 @@ uint32_t GetFileMetadataByExprRequest::read(::apache::thrift::protocol::TProtoco
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast688;
-          xfer += iprot->readI32(ecast688);
-          this->type = (FileMetadataExprType::type)ecast688;
+          int32_t ecast694;
+          xfer += iprot->readI32(ecast694);
+          this->type = (FileMetadataExprType::type)ecast694;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -16910,10 +16955,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 _iter689;
-    for (_iter689 = this->fileIds.begin(); _iter689 != this->fileIds.end(); ++_iter689)
+    std::vector<int64_t> ::const_iterator _iter695;
+    for (_iter695 = this->fileIds.begin(); _iter695 != this->fileIds.end(); ++_iter695)
     {
-      xfer += oprot->writeI64((*_iter689));
+      xfer += oprot->writeI64((*_iter695));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16947,19 +16992,19 @@ void swap(GetFileMetadataByExprRequest &a, GetFileMetadataByExprRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other690) {
-  fileIds = other690.fileIds;
-  expr = other690.expr;
-  doGetFooters = other690.doGetFooters;
-  type = other690.type;
-  __isset = other690.__isset;
+GetFileMetadataByExprRequest::GetFileMetadataByExprRequest(const GetFileMetadataByExprRequest& other696) {
+  fileIds = other696.fileIds;
+  expr = other696.expr;
+  doGetFooters = other696.doGetFooters;
+  type = other696.type;
+  __isset = other696.__isset;
 }
-GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other691) {
-  fileIds = other691.fileIds;
-  expr = other691.expr;
-  doGetFooters = other691.doGetFooters;
-  type = other691.type;
-  __isset = other691.__isset;
+GetFileMetadataByExprRequest& GetFileMetadataByExprRequest::operator=(const GetFileMetadataByExprRequest& other697) {
+  fileIds = other697.fileIds;
+  expr = other697.expr;
+  doGetFooters = other697.doGetFooters;
+  type = other697.type;
+  __isset = other697.__isset;
   return *this;
 }
 void GetFileMetadataByExprRequest::printTo(std::ostream& out) const {
@@ -17012,17 +17057,17 @@ uint32_t GetFileMetadataResult::read(::apache::thrift::protocol::TProtocol* ipro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->metadata.clear();
-            uint32_t _size692;
-            ::apache::thrift::protocol::TType _ktype693;
-            ::apache::thrift::protocol::TType _vtype694;
-            xfer += iprot->readMapBegin(_ktype693, _vtype694, _size692);
-            uint32_t _i696;
-            for (_i696 = 0; _i696 < _size692; ++_i696)
+            uint32_t _size698;
+            ::apache::thrift::protocol::TType _ktype699;
+            ::apache::thrift::protocol::TType _vtype700;
+            xfer += iprot->readMapBegin(_ktype699, _vtype700, _size698);
+            uint32_t _i702;
+            for (_i702 = 0; _i702 < _size698; ++_i702)
             {
-              int64_t _key697;
-              xfer += iprot->readI64(_key697);
-              std::string& _val698 = this->metadata[_key697];
-              xfer += iprot->readBinary(_val698);
+              int64_t _key703;
+              xfer += iprot->readI64(_key703);
+              std::string& _val704 = this->metadata[_key703];
+              xfer += iprot->readBinary(_val704);
             }
             xfer += iprot->readMapEnd();
           }
@@ -17063,11 +17108,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 _iter699;
-    for (_iter699 = this->metadata.begin(); _iter699 != this->metadata.end(); ++_iter699)
+    std::map<int64_t, std::string> ::const_iterator _iter705;
+    for (_iter705 = this->metadata.begin(); _iter705 != this->metadata.end(); ++_iter705)
     {
-      xfer += oprot->writeI64(_iter699->first);
-      xfer += oprot->writeBinary(_iter699->second);
+      xfer += oprot->writeI64(_iter705->first);
+      xfer += oprot->writeBinary(_iter705->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -17088,13 +17133,13 @@ void swap(GetFileMetadataResult &a, GetFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other700) {
-  metadata = other700.metadata;
-  isSupported = other700.isSupported;
+GetFileMetadataResult::GetFileMetadataResult(const GetFileMetadataResult& other706) {
+  metadata = other706.metadata;
+  isSupported = other706.isSupported;
 }
-GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other701) {
-  metadata = other701.metadata;
-  isSupported = other701.isSupported;
+GetFileMetadataResult& GetFileMetadataResult::operator=(const GetFileMetadataResult& other707) {
+  metadata = other707.metadata;
+  isSupported = other707.isSupported;
   return *this;
 }
 void GetFileMetadataResult::printTo(std::ostream& out) const {
@@ -17140,14 +17185,14 @@ uint32_t GetFileMetadataRequest::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 _size708;
+            ::apache::thrift::protocol::TType _etype711;
+            xfer += iprot->readListBegin(_etype711, _size708);
+            this->fileIds.resize(_size708);
+            uint32_t _i712;
+            for (_i712 = 0; _i712 < _size708; ++_i712)
             {
-              xfer += iprot->readI64(this->fileIds[_i706]);
+              xfer += iprot->readI64(this->fileIds[_i712]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17178,10 +17223,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 _iter707;
-    for (_iter707 = this->fileIds.begin(); _iter707 != this->fileIds.end(); ++_iter707)
+    std::vector<int64_t> ::const_iterator _iter713;
+    for (_iter713 = this->fileIds.begin(); _iter713 != this->fileIds.end(); ++_iter713)
     {
-      xfer += oprot->writeI64((*_iter707));
+      xfer += oprot->writeI64((*_iter713));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17197,11 +17242,11 @@ void swap(GetFileMetadataRequest &a, GetFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other708) {
-  fileIds = other708.fileIds;
+GetFileMetadataRequest::GetFileMetadataRequest(const GetFileMetadataRequest& other714) {
+  fileIds = other714.fileIds;
 }
-GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other709) {
-  fileIds = other709.fileIds;
+GetFileMetadataRequest& GetFileMetadataRequest::operator=(const GetFileMetadataRequest& other715) {
+  fileIds = other715.fileIds;
   return *this;
 }
 void GetFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17260,11 +17305,11 @@ void swap(PutFileMetadataResult &a, PutFileMetadataResult &b) {
   (void) b;
 }
 
-PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other710) {
-  (void) other710;
+PutFileMetadataResult::PutFileMetadataResult(const PutFileMetadataResult& other716) {
+  (void) other716;
 }
-PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other711) {
-  (void) other711;
+PutFileMetadataResult& PutFileMetadataResult::operator=(const PutFileMetadataResult& other717) {
+  (void) other717;
   return *this;
 }
 void PutFileMetadataResult::printTo(std::ostream& out) const {
@@ -17318,14 +17363,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size712;
-            ::apache::thrift::protocol::TType _etype715;
-            xfer += iprot->readListBegin(_etype715, _size712);
-            this->fileIds.resize(_size712);
-            uint32_t _i716;
-            for (_i716 = 0; _i716 < _size712; ++_i716)
+            uint32_t _size718;
+            ::apache::thrift::protocol::TType _etype721;
+            xfer += iprot->readListBegin(_etype721, _size718);
+            this->fileIds.resize(_size718);
+            uint32_t _i722;
+            for (_i722 = 0; _i722 < _size718; ++_i722)
             {
-              xfer += iprot->readI64(this->fileIds[_i716]);
+              xfer += iprot->readI64(this->fileIds[_i722]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17338,14 +17383,14 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->metadata.clear();
-            uint32_t _size717;
-            ::apache::thrift::protocol::TType _etype720;
-            xfer += iprot->readListBegin(_etype720, _size717);
-            this->metadata.resize(_size717);
-            uint32_t _i721;
-            for (_i721 = 0; _i721 < _size717; ++_i721)
+            uint32_t _size723;
+            ::apache::thrift::protocol::TType _etype726;
+            xfer += iprot->readListBegin(_etype726, _size723);
+            this->metadata.resize(_size723);
+            uint32_t _i727;
+            for (_i727 = 0; _i727 < _size723; ++_i727)
             {
-              xfer += iprot->readBinary(this->metadata[_i721]);
+              xfer += iprot->readBinary(this->metadata[_i727]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17356,9 +17401,9 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast722;
-          xfer += iprot->readI32(ecast722);
-          this->type = (FileMetadataExprType::type)ecast722;
+          int32_t ecast728;
+          xfer += iprot->readI32(ecast728);
+          this->type = (FileMetadataExprType::type)ecast728;
           this->__isset.type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17388,10 +17433,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 _iter723;
-    for (_iter723 = this->fileIds.begin(); _iter723 != this->fileIds.end(); ++_iter723)
+    std::vector<int64_t> ::const_iterator _iter729;
+    for (_iter729 = this->fileIds.begin(); _iter729 != this->fileIds.end(); ++_iter729)
     {
-      xfer += oprot->writeI64((*_iter723));
+      xfer += oprot->writeI64((*_iter729));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17400,10 +17445,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 _iter724;
-    for (_iter724 = this->metadata.begin(); _iter724 != this->metadata.end(); ++_iter724)
+    std::vector<std::string> ::const_iterator _iter730;
+    for (_iter730 = this->metadata.begin(); _iter730 != this->metadata.end(); ++_iter730)
     {
-      xfer += oprot->writeBinary((*_iter724));
+      xfer += oprot->writeBinary((*_iter730));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17427,17 +17472,17 @@ void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other725) {
-  fileIds = other725.fileIds;
-  metadata = other725.metadata;
-  type = other725.type;
-  __isset = other725.__isset;
+PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other731) {
+  fileIds = other731.fileIds;
+  metadata = other731.metadata;
+  type = other731.type;
+  __isset = other731.__isset;
 }
-PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other726) {
-  fileIds = other726.fileIds;
-  metadata = other726.metadata;
-  type = other726.type;
-  __isset = other726.__isset;
+PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other732) {
+  fileIds = other732.fileIds;
+  metadata = other732.metadata;
+  type = other732.type;
+  __isset = other732.__isset;
   return *this;
 }
 void PutFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17498,11 +17543,11 @@ void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) {
   (void) b;
 }
 
-ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other727) {
-  (void) other727;
+ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other733) {
+  (void) other733;
 }
-ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other728) {
-  (void) other728;
+ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other734) {
+  (void) other734;
   return *this;
 }
 void ClearFileMetadataResult::printTo(std::ostream& out) const {
@@ -17546,14 +17591,14 @@ uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size729;
-            ::apache::thrift::protocol::TType _etype732;
-            xfer += iprot->readListBegin(_etype732, _size729);
-            this->fileIds.resize(_size729);
-            uint32_t _i733;
-            for (_i733 = 0; _i733 < _size729; ++_i733)
+            uint32_t _size735;
+            ::apache::thrift::protocol::TType _etype738;
+            xfer += iprot->readListBegin(_etype738, _size735);
+            this->fileIds.resize(_size735);
+            uint32_t _i739;
+            for (_i739 = 0; _i739 < _size735; ++_i739)
             {
-              xfer += iprot->readI64(this->fileIds[_i733]);
+              xfer += iprot->readI64(this->fileIds[_i739]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17584,10 +17629,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 _iter734;
-    for (_iter734 = this->fileIds.begin(); _iter734 != this->fileIds.end(); ++_iter734)
+    std::vector<int64_t> ::const_iterator _iter740;
+    for (_iter740 = this->fileIds.begin(); _iter740 != this->fileIds.end(); ++_iter740)
     {
-      xfer += oprot->writeI64((*_iter734));
+      xfer += oprot->writeI64((*_iter740));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17603,11 +17648,11 @@ void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other735) {
-  fileIds = other735.fileIds;
+ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other741) {
+  fileIds = other741.fileIds;
 }
-ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other736) {
-  fileIds = other736.fileIds;
+ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other742) {
+  fileIds = other742.fileIds;
   return *this;
 }
 void ClearFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17689,11 +17734,11 @@ void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) {
   swap(a.isSupported, b.isSupported);
 }
 
-CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other737) {
-  isSupported = other737.isSupported;
+CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other743) {
+  isSupported = other743.isSupported;
 }
-CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other738) {
-  isSupported = other738.isSupported;
+CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other744) {
+  isSupported = other744.isSupported;
   return *this;
 }
 void CacheFileMetadataResult::printTo(std::ostream& out) const {
@@ -17834,19 +17879,19 @@ void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other739) {
-  dbName = other739.dbName;
-  tblName = other739.tblName;
-  partName = other739.partName;
-  isAllParts = other739.isAllParts;
-  __isset = other739.__isset;
+CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other745) {
+  dbName = other745.dbName;
+  tblName = other745.tblName;
+  partName = other745.partName;
+  isAllParts = other745.isAllParts;
+  __isset = other745.__isset;
 }
-CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other740) {
-  dbName = other740.dbName;
-  tblName = other740.tblName;
-  partName = other740.partName;
-  isAllParts = other740.isAllParts;
-  __isset = other740.__isset;
+CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other746) {
+  dbName = other746.dbName;
+  tblName = other746.tblName;
+  partName = other746.partName;
+  isAllParts = other746.isAllParts;
+  __isset = other746.__isset;
   return *this;
 }
 void CacheFileMetadataRequest::printTo(std::ostream& out) const {
@@ -17894,14 +17939,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size741;
-            ::apache::thrift::protocol::TType _etype744;
-            xfer += iprot->readListBegin(_etype744, _size741);
-            this->functions.resize(_size741);
-            uint32_t _i745;
-            for (_i745 = 0; _i745 < _size741; ++_i745)
+            uint32_t _size747;
+            ::apache::thrift::protocol::TType _etype750;
+            xfer += iprot->readListBegin(_etype750, _size747);
+            this->functions.resize(_size747);
+            uint32_t _i751;
+            for (_i751 = 0; _i751 < _size747; ++_i751)
             {
-              xfer += this->functions[_i745].read(iprot);
+              xfer += this->functions[_i751].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17931,10 +17976,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 _iter746;
-      for (_iter746 = this->functions.begin(); _iter746 != this->functions.end(); ++_iter746)
+      std::vector<Function> ::const_iterator _iter752;
+      for (_iter752 = this->functions.begin(); _iter752 != this->functions.end(); ++_iter752)
       {
-        xfer += (*_iter746).write(oprot);
+        xfer += (*_iter752).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -17951,13 +17996,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other747) {
-  functions = other747.functions;
-  __isset = other747.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other753) {
+  functions = other753.functions;
+  __isset = other753.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other748) {
-  functions = other748.functions;
-  __isset = other748.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other754) {
+  functions = other754.functions;
+  __isset = other754.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -18002,16 +18047,16 @@ uint32_t ClientCapabilities::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size749;
-            ::apache::thrift::protocol::TType _etype752;
-            xfer += iprot->readListBegin(_etype752, _size749);
-            this->values.resize(_size749);
-            uint32_t _i753;
-            for (_i753 = 0; _i753 < _size749; ++_i753)
+            uint32_t _size755;
+            ::apache::thrift::protocol::TType _etype758;
+            xfer += iprot->readListBegin(_etype758, _size755);
+            this->values.resize(_size755);
+            uint32_t _i759;
+            for (_i759 = 0; _i759 < _size755; ++_i759)
             {
-              int32_t ecast754;
-              xfer += iprot->readI32(ecast754);
-              this->values[_i753] = (ClientCapability::type)ecast754;
+              int32_t ecast760;
+              xfer += iprot->readI32(ecast760);
+              this->values[_i759] = (ClientCapability::type)ecast760;
             }
             xfer += iprot->readListEnd();
           }
@@ -18042,10 +18087,10 @@ uint32_t ClientCapabilities::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->values.size()));
-    std::vector<ClientCapability::type> ::const_iterator _iter755;
-    for (_iter755 = this->values.begin(); _iter755 != this->values.end(); ++_iter755)
+    std::vector<ClientCapability::type> ::const_iterator _iter761;
+    for (_iter761 = this->values.begin(); _iter761 != this->values.end(); ++_iter761)
     {
-      xfer += oprot->writeI32((int32_t)(*_iter755));
+      xfer += oprot->writeI32((int32_t)(*_iter761));
     }
     xfer += oprot->writeListEnd();
   }
@@ -18061,11 +18106,11 @@ void swap(ClientCapabilities &a, ClientCapabilities &b) {
   swap(a.values, b.values);
 }
 
-ClientCapabilities::ClientCapabilities(const ClientCapabilities& other756) {
-  values = other756.values;
+ClientCapabilities::ClientCapabilities(const ClientCapabilities& other762) {
+  values = other762.values;
 }
-ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other757) {
-  values = other757.values;
+ClientCapabilities& ClientCapabilities::operator=(const ClientCapabilities& other763) {
+  values = other763.values;
   return *this;
 }
 void ClientCapabilities::printTo(std::ostream& out) const {
@@ -18187,17 +18232,17 @@ void swap(GetTableRequest &a, GetTableRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetTableRequest::GetTableRequest(const GetTableRequest& other758) {
-  dbName = other758.dbName;
-  tblName = other758.tblName;
-  capabilities = other758.capabilities;
-  __isset = other758.__isset;
+GetTableRequest::GetTableRequest(const GetTableRequest& other764) {
+  dbName = other764.dbName;
+  tblName = other764.tblName;
+  capabilities = other764.capabilities;
+  __isset = other764.__isset;
 }
-GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other759) {
-  dbName = other759.dbName;
-  tblName = other759.tblName;
-  capabilities = other759.capabilities;
-  __isset = other759.__isset;
+GetTableRequest& GetTableRequest::operator=(const GetTableRequest& other765) {
+  dbName = other765.dbName;
+  tblName = other765.tblName;
+  capabilities = other765.capabilities;
+  __isset = other765.__isset;
   return *this;
 }
 void GetTableRequest::printTo(std::ostream& out) const {
@@ -18281,11 +18326,11 @@ void swap(GetTableResult &a, GetTableResult &b) {
   swap(a.table, b.table);
 }
 
-GetTableResult::GetTableResult(const GetTableResult& other760) {
-  table = other760.table;
+GetTableResult::GetTableResult(const GetTableResult& other766) {
+  table = other766.table;
 }
-GetTableResult& GetTableResult::operator=(const GetTableResult& other761) {
-  table = other761.table;
+GetTableResult& GetTableResult::operator=(const GetTableResult& other767) {
+  table = other767.table;
   return *this;
 }
 void GetTableResult::printTo(std::ostream& out) const {
@@ -18348,14 +18393,14 @@ uint32_t GetTablesRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tblNames.clear();
-            uint32_t _size762;
-            ::apache::thrift::protocol::TType _etype765;
-            xfer += iprot->readListBegin(_etype765, _size762);
-            this->tblNames.resize(_size762);
-            uint32_t _i766;
-            for (_i766 = 0; _i766 < _size762; ++_i766)
+            uint32_t _size768;
+            ::apache::thrift::protocol::TType _etype771;
+            xfer += iprot->readListBegin(_etype771, _size768);
+            this->tblNames.resize(_size768);
+            uint32_t _i772;
+            for (_i772 = 0; _i772 < _size768; ++_i772)
             {
-              xfer += iprot->readString(this->tblNames[_i766]);
+              xfer += iprot->readString(this->tblNames[_i772]);
             }
             xfer += iprot->readListEnd();
           }
@@ -18399,10 +18444,10 @@ uint32_t GetTablesRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("tblNames", ::apache::thrift::protocol::T_LIST, 2);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tblNames.size()));
-      std::vector<std::string> ::const_iterator _iter767;
-      for (_iter767 = this->tblNames.begin(); _iter767 != this->tblNames.end(); ++_iter767)
+      std::vector<std::string> ::const_iterator _iter773;
+      for (_iter773 = this->tblNames.begin(); _iter773 != this->tblNames.end(); ++_iter773)
       {
-        xfer += oprot->writeString((*_iter767));
+        xfer += oprot->writeString((*_iter773));
       }
       xfer += oprot->writeListEnd();
     }
@@ -18426,17 +18471,17 @@ void swap(GetTablesRequest &a, GetTablesRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetTablesRequest::GetTablesRequest(const GetTablesRequest& other768) {
-  dbName = other768.dbName;
-  tblNames = other768.tblNames;
-  capabilities = other768.capabilities;
-  __isset = other768.__isset;
+GetTablesRequest::GetTablesRequest(const GetTablesRequest& other774) {
+  dbName = other774.dbName;
+  tblNames = other774.tblNames;
+  capabilities = other774.capabilities;
+  __isset = other774.__isset;
 }
-GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other769) {
-  dbName = other769.dbName;
-  tblNames = other769.tblNames;
-  capabilities = other769.capabilities;
-  __isset = other769.__isset;
+GetTablesRequest& GetTablesRequest::operator=(const GetTablesRequest& other775) {
+  dbName = other775.dbName;
+  tblNames = other775.tblNames;
+  capabilities = other775.capabilities;
+  __isset = other775.__isset;
   return *this;
 }
 void GetTablesRequest::printTo(std::ostream& out) const {
@@ -18483,14 +18528,14 @@ uint32_t GetTablesResult::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tables.clear();
-            uint32_t _size770;
-            ::apache::thrift::protocol::TType _etype773;
-            xfer += iprot->readListBegin(_etype773, _size770);
-            this->tables.resize(_size770);
-            uint32_t _i774;
-            for (_i774 = 0; _i774 < _size770; ++_i774)
+            uint32_t _size776;
+            ::apache::thrift::protocol::TType _etype779;
+            xfer += iprot->readListBegin(_etype779, _size776);
+            this->tables.resize(_size776);
+            uint32_t _i780;
+            for (_i780 = 0; _i780 < _size776; ++_i780)
             {
-              xfer += this->tables[_i774].read(iprot);
+              xfer += this->tables[_i780].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18521,10 +18566,10 @@ uint32_t GetTablesResult::write(::apache::thrift::protocol::TProtocol* oprot) co
   xfer += oprot->writeFieldBegin("tables", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tables.size()));
-    std::vector<Table> ::const_iterator _iter775;
-    for (_iter775 = this->tables.begin(); _iter775 != this->tables.end(); ++_iter775)
+    std::vector<Table> ::const_iterator _iter781;
+    for (_iter781 = this->tables.begin(); _iter781 != this->tables.end(); ++_iter781)
     {
-      xfer += (*_iter775).write(oprot);
+      xfer += (*_iter781).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18540,11 +18585,11 @@ void swap(GetTablesResult &a, GetTablesResult &b) {
   swap(a.tables, b.tables);
 }
 
-GetTablesResult::GetTablesResult(const GetTablesResult& other776) {
-  tables = other776.tables;
+GetTablesResult::GetTablesResult(const GetTablesResult& other782) {
+  tables = other782.tables;
 }
-GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other777) {
-  tables = other777.tables;
+GetTablesResult& GetTablesResult::operator=(const GetTablesResult& other783) {
+  tables = other783.tables;
   return *this;
 }
 void GetTablesResult::printTo(std::ostream& out) const {
@@ -18686,19 +18731,19 @@ void swap(TableMeta &a, TableMeta &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableMeta::TableMeta(const TableMeta& other778) {
-  dbName = other778.dbName;
-  tableName = other778.tableName;
-  tableType = other778.tableType;
-  comments = other778.comments;
-  __isset = other778.__isset;
+TableMeta::TableMeta(const TableMeta& other784) {
+  dbName = other784.dbName;
+  tableName = other784.tableName;
+  tableType = other784.tableType;
+  comments = other784.comments;
+  __isset = other784.__isset;
 }
-TableMeta& TableMeta::operator=(const TableMeta& other779) {
-  dbName = other779.dbName;
-  tableName = other779.tableName;
-  tableType = other779.tableType;
-  comments = other779.comments;
-  __isset = other779.__isset;
+TableMeta& TableMeta::operator=(const TableMeta& other785) {
+  dbName = other785.dbName;
+  tableName = other785.tableName;
+  tableType = other785.tableType;
+  comments = other785.comments;
+  __isset = other785.__isset;
   return *this;
 }
 void TableMeta::printTo(std::ostream& out) const {
@@ -18781,13 +18826,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other780) : TException() {
-  message = other780.message;
-  __isset = other780.__isset;
+MetaException::MetaException(const MetaException& other786) : TException() {
+  message = other786.message;
+  __isset = other786.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other781) {
-  message = other781.message;
-  __isset = other781.__isset;
+MetaException& MetaException::operator=(const MetaException& other787) {
+  message = other787.message;
+  __isset = other787.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -18878,13 +18923,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other782) : TException() {
-  message = other782.message;
-  __isset = other782.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other788) : TException() {
+  message = other788.message;
+  __isset = other788.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other783) {
-  message = other783.message;
-  __isset = other783.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other789) {
+  message = other789.message;
+  __isset = other789.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -18975,13 +19020,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other784) : TException() {
-  message = other784.message;
-  __isset = other784.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other790) : TException() {
+  message = other790.message;
+  __isset = other790.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other785) {
-  message = other785.message;
-  __isset = other785.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other791) {
+  message = other791.message;
+  __isset = other791.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -19072,13 +19117,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other786) : TException() {
-  message = other786.message;
-  __isset = other786.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other792) : TException() {
+  message = other792.message;
+  __isset = other792.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other787) {
-  message = other787.message;
-  __isset = other787.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other793) {
+  message = other793.message;
+  __isset = other793.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -19169,13 +19214,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other788) : TException() {
-  message = other788.message;
-  __isset = other788.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other794) : TException() {
+  message = other794.message;
+  __isset = other794.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other789) {
-  message = other789.message;
-  __isset = other789.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other795) {
+  message = other795.message;
+  __isset = other795.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -19266,13 +19311,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other790) : TException() {
-  message = other790.message;
-  __isset = other790.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other796) : TException() {
+  message = other796.message;
+  __isset = other796.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other791) {
-  message = other791.message;
-  __isset = other791.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other797) {
+  message = other797.message;
+  __isset = other797.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -19363,13 +19408,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other792) : TException() {
-  message = other792.message;
-  __isset = other792.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other798) : TException() {
+  message = other798.message;
+  __isset = other798.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other793) {
-  message = other793.message;
-  __isset = other793.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other799) {
+  message = other799.message;
+  __isset = other799.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -19460,13 +19505,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other794) : TException() {
-  message = other794.message;
-  __isset = other794.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other800) : TException() {
+  message = other800.message;
+  __isset = other800.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other795) {
-  message = other795.message;
-  __isset = other795.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other801) {
+  message = other801.message;
+  __isset = other801.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -19557,13 +19602,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other796) : TException() {
-  message = other796.message;
-  __isset = other796.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other802) : TException() {
+  message = other802.message;
+  __isset = other802.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other797) {
-  message = other797.message;
-  __isset = other797.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other803) {
+  message = other803.message;
+  __isset = other803.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -19654,13 +19699,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other798) : TException() {
-  message = other798.message;
-  __isset = other798.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other804) : TException() {
+  message = other804.message;
+  __isset = other804.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other799) {
-  message = other799.message;
-  __isset = other799.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other805) {
+  message = other805.message;
+  __isset = other805.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -19751,13 +19796,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other800) : TException() {
-  message = other800.message;
-  __isset = other800.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other806) : TException() {
+  message = other806.message;
+  __isset = other806.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other801) {
-  message = other801.message;
-  __isset = other801.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other807) {
+  message = other807.message;
+  __isset = other807.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -19848,13 +19893,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other802) : TException() {
-  message = other802.message;
-  __isset = other802.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other808) : TException() {
+  message = other808.message;
+  __isset = other808.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other803) {
-  message = other803.message;
-  __isset = other803.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other809) {
+  message = other809.message;
+  __isset = other809.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -19945,13 +19990,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other804) : TException() {
-  message = other804.message;
-  __isset = other804.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other810) : TException() {
+  message = other810.message;
+  __isset = other810.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other805) {
-  message = other805.message;
-  __isset = other805.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other811) {
+  message = other811.message;
+  __isset = other811.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -20042,13 +20087,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other806) : TException() {
-  message = other806.message;
-  __isset = other806.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other812) : TException() {
+  message = other812.message;
+  __isset = other812.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other807) {
-  message = other807.message;
-  __isset = other807.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other813) {
+  message = other813.message;
+  __isset = other813.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -20139,13 +20184,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other808) : TException() {
-  message = other808.message;
-  __isset = other808.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other814) : TException() {
+  message = other814.message;
+  __isset = other814.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other809) {
-  message = other809.message;
-  __isset = other809.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other815) {
+  message = other815.message;
+  __isset = other815.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -20236,13 +20281,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other810) : TException() {
-  message = other810.message;
-  __isset = other810.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other816) : TException() {
+  message = other816.message;
+  __isset = other816.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other811) {
-  message = other811.message;
-  __isset = other811.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other817) {
+  message = other817.message;
+  __isset = other817.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 0104f6e..4d5da71 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -6546,6 +6546,10 @@ inline std::ostream& operator<<(std::ostream& out, const CurrentNotificationEven
   return out;
 }
 
+typedef struct _InsertEventRequestData__isset {
+  _InsertEventRequestData__isset() : filesAddedChecksum(false) {}
+  bool filesAddedChecksum :1;
+} _InsertEventRequestData__isset;
 
 class InsertEventRequestData {
  public:
@@ -6557,13 +6561,22 @@ class InsertEventRequestData {
 
   virtual ~InsertEventRequestData() throw();
   std::vector<std::string>  filesAdded;
+  std::vector<std::string>  filesAddedChecksum;
+
+  _InsertEventRequestData__isset __isset;
 
   void __set_filesAdded(const std::vector<std::string> & val);
 
+  void __set_filesAddedChecksum(const std::vector<std::string> & val);
+
   bool operator == (const InsertEventRequestData & rhs) const
   {
     if (!(filesAdded == rhs.filesAdded))
       return false;
+    if (__isset.filesAddedChecksum != rhs.__isset.filesAddedChecksum)
+      return false;
+    else if (__isset.filesAddedChecksum && !(filesAddedChecksum == rhs.filesAddedChecksum))
+      return false;
     return true;
   }
   bool operator != (const InsertEventRequestData &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 ca274e6..0da09bf 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 _list626 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list626.size);
-                long _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list634.size);
+                long _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem627 = iprot.readI64();
-                  struct.fileIds.add(_elem627);
+                  _elem635 = iprot.readI64();
+                  struct.fileIds.add(_elem635);
                 }
                 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 _iter629 : struct.fileIds)
+          for (long _iter637 : struct.fileIds)
           {
-            oprot.writeI64(_iter629);
+            oprot.writeI64(_iter637);
           }
           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 _iter630 : struct.fileIds)
+        for (long _iter638 : struct.fileIds)
         {
-          oprot.writeI64(_iter630);
+          oprot.writeI64(_iter638);
         }
       }
     }
@@ -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 _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)
+        org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list639.size);
+        long _elem640;
+        for (int _i641 = 0; _i641 < _list639.size; ++_i641)
         {
-          _elem632 = iprot.readI64();
-          struct.fileIds.add(_elem632);
+          _elem640 = iprot.readI64();
+          struct.fileIds.add(_elem640);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index 5fc7d56..81534fe 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list642.size);
-                ClientCapability _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list650.size);
+                ClientCapability _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem643 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem643);
+                  _elem651 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem651);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter645 : struct.values)
+          for (ClientCapability _iter653 : struct.values)
           {
-            oprot.writeI32(_iter645.getValue());
+            oprot.writeI32(_iter653.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter646 : struct.values)
+        for (ClientCapability _iter654 : struct.values)
         {
-          oprot.writeI32(_iter646.getValue());
+          oprot.writeI32(_iter654.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ public class ClientCapabilities implements org.apache.thrift.TBase<ClientCapabil
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list647.size);
-        ClientCapability _elem648;
-        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list655.size);
+        ClientCapability _elem656;
+        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
         {
-          _elem648 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem648);
+          _elem656 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem656);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 6772338..7bb10b3 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 _list566 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list566.size);
-                String _elem567;
-                for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list574.size);
+                String _elem575;
+                for (int _i576 = 0; _i576 < _list574.size; ++_i576)
                 {
-                  _elem567 = iprot.readString();
-                  struct.partitionVals.add(_elem567);
+                  _elem575 = iprot.readString();
+                  struct.partitionVals.add(_elem575);
                 }
                 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 _iter569 : struct.partitionVals)
+            for (String _iter577 : struct.partitionVals)
             {
-              oprot.writeString(_iter569);
+              oprot.writeString(_iter577);
             }
             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 _iter570 : struct.partitionVals)
+          for (String _iter578 : struct.partitionVals)
           {
-            oprot.writeString(_iter570);
+            oprot.writeString(_iter578);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          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)
+          org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list579.size);
+          String _elem580;
+          for (int _i581 = 0; _i581 < _list579.size; ++_i581)
           {
-            _elem572 = iprot.readString();
-            struct.partitionVals.add(_elem572);
+            _elem580 = iprot.readString();
+            struct.partitionVals.add(_elem580);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 f427a3a..49a1be2 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 _list634 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list634.size);
-                Function _elem635;
-                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
+                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list642.size);
+                Function _elem643;
+                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
                 {
-                  _elem635 = new Function();
-                  _elem635.read(iprot);
-                  struct.functions.add(_elem635);
+                  _elem643 = new Function();
+                  _elem643.read(iprot);
+                  struct.functions.add(_elem643);
                 }
                 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 _iter637 : struct.functions)
+            for (Function _iter645 : struct.functions)
             {
-              _iter637.write(oprot);
+              _iter645.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 _iter638 : struct.functions)
+          for (Function _iter646 : struct.functions)
           {
-            _iter638.write(oprot);
+            _iter646.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 _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)
+          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list647.size);
+          Function _elem648;
+          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
           {
-            _elem640 = new Function();
-            _elem640.read(iprot);
-            struct.functions.add(_elem640);
+            _elem648 = new Function();
+            _elem648.read(iprot);
+            struct.functions.add(_elem648);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 1ea90a0..20b82cf 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 _list584 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list584.size);
-                long _elem585;
-                for (int _i586 = 0; _i586 < _list584.size; ++_i586)
+                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)
                 {
-                  _elem585 = iprot.readI64();
-                  struct.fileIds.add(_elem585);
+                  _elem593 = iprot.readI64();
+                  struct.fileIds.add(_elem593);
                 }
                 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 _iter587 : struct.fileIds)
+          for (long _iter595 : struct.fileIds)
           {
-            oprot.writeI64(_iter587);
+            oprot.writeI64(_iter595);
           }
           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 _iter588 : struct.fileIds)
+        for (long _iter596 : struct.fileIds)
         {
-          oprot.writeI64(_iter588);
+          oprot.writeI64(_iter596);
         }
       }
       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 _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)
+        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)
         {
-          _elem590 = iprot.readI64();
-          struct.fileIds.add(_elem590);
+          _elem598 = iprot.readI64();
+          struct.fileIds.add(_elem598);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 609abc5..9975dfc 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 _map574 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map574.size);
-                long _key575;
-                MetadataPpdResult _val576;
-                for (int _i577 = 0; _i577 < _map574.size; ++_i577)
+                org.apache.thrift.protocol.TMap _map582 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map582.size);
+                long _key583;
+                MetadataPpdResult _val584;
+                for (int _i585 = 0; _i585 < _map582.size; ++_i585)
                 {
-                  _key575 = iprot.readI64();
-                  _val576 = new MetadataPpdResult();
-                  _val576.read(iprot);
-                  struct.metadata.put(_key575, _val576);
+                  _key583 = iprot.readI64();
+                  _val584 = new MetadataPpdResult();
+                  _val584.read(iprot);
+                  struct.metadata.put(_key583, _val584);
                 }
                 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> _iter578 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter586 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter578.getKey());
-            _iter578.getValue().write(oprot);
+            oprot.writeI64(_iter586.getKey());
+            _iter586.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> _iter579 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter587 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter579.getKey());
-          _iter579.getValue().write(oprot);
+          oprot.writeI64(_iter587.getKey());
+          _iter587.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 _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)
+        org.apache.thrift.protocol.TMap _map588 = 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*_map588.size);
+        long _key589;
+        MetadataPpdResult _val590;
+        for (int _i591 = 0; _i591 < _map588.size; ++_i591)
         {
-          _key581 = iprot.readI64();
-          _val582 = new MetadataPpdResult();
-          _val582.read(iprot);
-          struct.metadata.put(_key581, _val582);
+          _key589 = iprot.readI64();
+          _val590 = new MetadataPpdResult();
+          _val590.read(iprot);
+          struct.metadata.put(_key589, _val590);
         }
       }
       struct.setMetadataIsSet(true);


[3/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 0087205..9bfc2b2 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -11032,14 +11032,14 @@ class ThriftHiveMetastore_get_databases_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)
+            $_size597 = 0;
+            $_etype600 = 0;
+            $xfer += $input->readListBegin($_etype600, $_size597);
+            for ($_i601 = 0; $_i601 < $_size597; ++$_i601)
             {
-              $elem595 = null;
-              $xfer += $input->readString($elem595);
-              $this->success []= $elem595;
+              $elem602 = null;
+              $xfer += $input->readString($elem602);
+              $this->success []= $elem602;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11075,9 +11075,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter596)
+          foreach ($this->success as $iter603)
           {
-            $xfer += $output->writeString($iter596);
+            $xfer += $output->writeString($iter603);
           }
         }
         $output->writeListEnd();
@@ -11208,14 +11208,14 @@ class ThriftHiveMetastore_get_all_databases_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)
+            $_size604 = 0;
+            $_etype607 = 0;
+            $xfer += $input->readListBegin($_etype607, $_size604);
+            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
             {
-              $elem602 = null;
-              $xfer += $input->readString($elem602);
-              $this->success []= $elem602;
+              $elem609 = null;
+              $xfer += $input->readString($elem609);
+              $this->success []= $elem609;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11251,9 +11251,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter603)
+          foreach ($this->success as $iter610)
           {
-            $xfer += $output->writeString($iter603);
+            $xfer += $output->writeString($iter610);
           }
         }
         $output->writeListEnd();
@@ -12254,18 +12254,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size604 = 0;
-            $_ktype605 = 0;
-            $_vtype606 = 0;
-            $xfer += $input->readMapBegin($_ktype605, $_vtype606, $_size604);
-            for ($_i608 = 0; $_i608 < $_size604; ++$_i608)
+            $_size611 = 0;
+            $_ktype612 = 0;
+            $_vtype613 = 0;
+            $xfer += $input->readMapBegin($_ktype612, $_vtype613, $_size611);
+            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
             {
-              $key609 = '';
-              $val610 = new \metastore\Type();
-              $xfer += $input->readString($key609);
-              $val610 = new \metastore\Type();
-              $xfer += $val610->read($input);
-              $this->success[$key609] = $val610;
+              $key616 = '';
+              $val617 = new \metastore\Type();
+              $xfer += $input->readString($key616);
+              $val617 = new \metastore\Type();
+              $xfer += $val617->read($input);
+              $this->success[$key616] = $val617;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -12301,10 +12301,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter611 => $viter612)
+          foreach ($this->success as $kiter618 => $viter619)
           {
-            $xfer += $output->writeString($kiter611);
-            $xfer += $viter612->write($output);
+            $xfer += $output->writeString($kiter618);
+            $xfer += $viter619->write($output);
           }
         }
         $output->writeMapEnd();
@@ -12508,15 +12508,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size613 = 0;
-            $_etype616 = 0;
-            $xfer += $input->readListBegin($_etype616, $_size613);
-            for ($_i617 = 0; $_i617 < $_size613; ++$_i617)
+            $_size620 = 0;
+            $_etype623 = 0;
+            $xfer += $input->readListBegin($_etype623, $_size620);
+            for ($_i624 = 0; $_i624 < $_size620; ++$_i624)
             {
-              $elem618 = null;
-              $elem618 = new \metastore\FieldSchema();
-              $xfer += $elem618->read($input);
-              $this->success []= $elem618;
+              $elem625 = null;
+              $elem625 = new \metastore\FieldSchema();
+              $xfer += $elem625->read($input);
+              $this->success []= $elem625;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12568,9 +12568,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter619)
+          foreach ($this->success as $iter626)
           {
-            $xfer += $iter619->write($output);
+            $xfer += $iter626->write($output);
           }
         }
         $output->writeListEnd();
@@ -12812,15 +12812,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size620 = 0;
-            $_etype623 = 0;
-            $xfer += $input->readListBegin($_etype623, $_size620);
-            for ($_i624 = 0; $_i624 < $_size620; ++$_i624)
+            $_size627 = 0;
+            $_etype630 = 0;
+            $xfer += $input->readListBegin($_etype630, $_size627);
+            for ($_i631 = 0; $_i631 < $_size627; ++$_i631)
             {
-              $elem625 = null;
-              $elem625 = new \metastore\FieldSchema();
-              $xfer += $elem625->read($input);
-              $this->success []= $elem625;
+              $elem632 = null;
+              $elem632 = new \metastore\FieldSchema();
+              $xfer += $elem632->read($input);
+              $this->success []= $elem632;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12872,9 +12872,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter626)
+          foreach ($this->success as $iter633)
           {
-            $xfer += $iter626->write($output);
+            $xfer += $iter633->write($output);
           }
         }
         $output->writeListEnd();
@@ -13088,15 +13088,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size627 = 0;
-            $_etype630 = 0;
-            $xfer += $input->readListBegin($_etype630, $_size627);
-            for ($_i631 = 0; $_i631 < $_size627; ++$_i631)
+            $_size634 = 0;
+            $_etype637 = 0;
+            $xfer += $input->readListBegin($_etype637, $_size634);
+            for ($_i638 = 0; $_i638 < $_size634; ++$_i638)
             {
-              $elem632 = null;
-              $elem632 = new \metastore\FieldSchema();
-              $xfer += $elem632->read($input);
-              $this->success []= $elem632;
+              $elem639 = null;
+              $elem639 = new \metastore\FieldSchema();
+              $xfer += $elem639->read($input);
+              $this->success []= $elem639;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13148,9 +13148,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter633)
+          foreach ($this->success as $iter640)
           {
-            $xfer += $iter633->write($output);
+            $xfer += $iter640->write($output);
           }
         }
         $output->writeListEnd();
@@ -13392,15 +13392,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size634 = 0;
-            $_etype637 = 0;
-            $xfer += $input->readListBegin($_etype637, $_size634);
-            for ($_i638 = 0; $_i638 < $_size634; ++$_i638)
+            $_size641 = 0;
+            $_etype644 = 0;
+            $xfer += $input->readListBegin($_etype644, $_size641);
+            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
             {
-              $elem639 = null;
-              $elem639 = new \metastore\FieldSchema();
-              $xfer += $elem639->read($input);
-              $this->success []= $elem639;
+              $elem646 = null;
+              $elem646 = new \metastore\FieldSchema();
+              $xfer += $elem646->read($input);
+              $this->success []= $elem646;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13452,9 +13452,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter640)
+          foreach ($this->success as $iter647)
           {
-            $xfer += $iter640->write($output);
+            $xfer += $iter647->write($output);
           }
         }
         $output->writeListEnd();
@@ -14062,15 +14062,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size641 = 0;
-            $_etype644 = 0;
-            $xfer += $input->readListBegin($_etype644, $_size641);
-            for ($_i645 = 0; $_i645 < $_size641; ++$_i645)
+            $_size648 = 0;
+            $_etype651 = 0;
+            $xfer += $input->readListBegin($_etype651, $_size648);
+            for ($_i652 = 0; $_i652 < $_size648; ++$_i652)
             {
-              $elem646 = null;
-              $elem646 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem646->read($input);
-              $this->primaryKeys []= $elem646;
+              $elem653 = null;
+              $elem653 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem653->read($input);
+              $this->primaryKeys []= $elem653;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14080,15 +14080,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size647 = 0;
-            $_etype650 = 0;
-            $xfer += $input->readListBegin($_etype650, $_size647);
-            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
+            $_size654 = 0;
+            $_etype657 = 0;
+            $xfer += $input->readListBegin($_etype657, $_size654);
+            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
             {
-              $elem652 = null;
-              $elem652 = new \metastore\SQLForeignKey();
-              $xfer += $elem652->read($input);
-              $this->foreignKeys []= $elem652;
+              $elem659 = null;
+              $elem659 = new \metastore\SQLForeignKey();
+              $xfer += $elem659->read($input);
+              $this->foreignKeys []= $elem659;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14124,9 +14124,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter653)
+          foreach ($this->primaryKeys as $iter660)
           {
-            $xfer += $iter653->write($output);
+            $xfer += $iter660->write($output);
           }
         }
         $output->writeListEnd();
@@ -14141,9 +14141,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter654)
+          foreach ($this->foreignKeys as $iter661)
           {
-            $xfer += $iter654->write($output);
+            $xfer += $iter661->write($output);
           }
         }
         $output->writeListEnd();
@@ -15489,14 +15489,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size655 = 0;
-            $_etype658 = 0;
-            $xfer += $input->readListBegin($_etype658, $_size655);
-            for ($_i659 = 0; $_i659 < $_size655; ++$_i659)
+            $_size662 = 0;
+            $_etype665 = 0;
+            $xfer += $input->readListBegin($_etype665, $_size662);
+            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
             {
-              $elem660 = null;
-              $xfer += $input->readString($elem660);
-              $this->success []= $elem660;
+              $elem667 = null;
+              $xfer += $input->readString($elem667);
+              $this->success []= $elem667;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15532,9 +15532,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter661)
+          foreach ($this->success as $iter668)
           {
-            $xfer += $output->writeString($iter661);
+            $xfer += $output->writeString($iter668);
           }
         }
         $output->writeListEnd();
@@ -15736,14 +15736,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size662 = 0;
-            $_etype665 = 0;
-            $xfer += $input->readListBegin($_etype665, $_size662);
-            for ($_i666 = 0; $_i666 < $_size662; ++$_i666)
+            $_size669 = 0;
+            $_etype672 = 0;
+            $xfer += $input->readListBegin($_etype672, $_size669);
+            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
             {
-              $elem667 = null;
-              $xfer += $input->readString($elem667);
-              $this->success []= $elem667;
+              $elem674 = null;
+              $xfer += $input->readString($elem674);
+              $this->success []= $elem674;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15779,9 +15779,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter668)
+          foreach ($this->success as $iter675)
           {
-            $xfer += $output->writeString($iter668);
+            $xfer += $output->writeString($iter675);
           }
         }
         $output->writeListEnd();
@@ -15886,14 +15886,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size669 = 0;
-            $_etype672 = 0;
-            $xfer += $input->readListBegin($_etype672, $_size669);
-            for ($_i673 = 0; $_i673 < $_size669; ++$_i673)
+            $_size676 = 0;
+            $_etype679 = 0;
+            $xfer += $input->readListBegin($_etype679, $_size676);
+            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
             {
-              $elem674 = null;
-              $xfer += $input->readString($elem674);
-              $this->tbl_types []= $elem674;
+              $elem681 = null;
+              $xfer += $input->readString($elem681);
+              $this->tbl_types []= $elem681;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15931,9 +15931,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter675)
+          foreach ($this->tbl_types as $iter682)
           {
-            $xfer += $output->writeString($iter675);
+            $xfer += $output->writeString($iter682);
           }
         }
         $output->writeListEnd();
@@ -16010,15 +16010,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size676 = 0;
-            $_etype679 = 0;
-            $xfer += $input->readListBegin($_etype679, $_size676);
-            for ($_i680 = 0; $_i680 < $_size676; ++$_i680)
+            $_size683 = 0;
+            $_etype686 = 0;
+            $xfer += $input->readListBegin($_etype686, $_size683);
+            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
             {
-              $elem681 = null;
-              $elem681 = new \metastore\TableMeta();
-              $xfer += $elem681->read($input);
-              $this->success []= $elem681;
+              $elem688 = null;
+              $elem688 = new \metastore\TableMeta();
+              $xfer += $elem688->read($input);
+              $this->success []= $elem688;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16054,9 +16054,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter682)
+          foreach ($this->success as $iter689)
           {
-            $xfer += $iter682->write($output);
+            $xfer += $iter689->write($output);
           }
         }
         $output->writeListEnd();
@@ -16212,14 +16212,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size683 = 0;
-            $_etype686 = 0;
-            $xfer += $input->readListBegin($_etype686, $_size683);
-            for ($_i687 = 0; $_i687 < $_size683; ++$_i687)
+            $_size690 = 0;
+            $_etype693 = 0;
+            $xfer += $input->readListBegin($_etype693, $_size690);
+            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
             {
-              $elem688 = null;
-              $xfer += $input->readString($elem688);
-              $this->success []= $elem688;
+              $elem695 = null;
+              $xfer += $input->readString($elem695);
+              $this->success []= $elem695;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16255,9 +16255,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter689)
+          foreach ($this->success as $iter696)
           {
-            $xfer += $output->writeString($iter689);
+            $xfer += $output->writeString($iter696);
           }
         }
         $output->writeListEnd();
@@ -16572,14 +16572,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size690 = 0;
-            $_etype693 = 0;
-            $xfer += $input->readListBegin($_etype693, $_size690);
-            for ($_i694 = 0; $_i694 < $_size690; ++$_i694)
+            $_size697 = 0;
+            $_etype700 = 0;
+            $xfer += $input->readListBegin($_etype700, $_size697);
+            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
             {
-              $elem695 = null;
-              $xfer += $input->readString($elem695);
-              $this->tbl_names []= $elem695;
+              $elem702 = null;
+              $xfer += $input->readString($elem702);
+              $this->tbl_names []= $elem702;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16612,9 +16612,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter696)
+          foreach ($this->tbl_names as $iter703)
           {
-            $xfer += $output->writeString($iter696);
+            $xfer += $output->writeString($iter703);
           }
         }
         $output->writeListEnd();
@@ -16679,15 +16679,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size697 = 0;
-            $_etype700 = 0;
-            $xfer += $input->readListBegin($_etype700, $_size697);
-            for ($_i701 = 0; $_i701 < $_size697; ++$_i701)
+            $_size704 = 0;
+            $_etype707 = 0;
+            $xfer += $input->readListBegin($_etype707, $_size704);
+            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
             {
-              $elem702 = null;
-              $elem702 = new \metastore\Table();
-              $xfer += $elem702->read($input);
-              $this->success []= $elem702;
+              $elem709 = null;
+              $elem709 = new \metastore\Table();
+              $xfer += $elem709->read($input);
+              $this->success []= $elem709;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16715,9 +16715,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter703)
+          foreach ($this->success as $iter710)
           {
-            $xfer += $iter703->write($output);
+            $xfer += $iter710->write($output);
           }
         }
         $output->writeListEnd();
@@ -17383,14 +17383,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size704 = 0;
-            $_etype707 = 0;
-            $xfer += $input->readListBegin($_etype707, $_size704);
-            for ($_i708 = 0; $_i708 < $_size704; ++$_i708)
+            $_size711 = 0;
+            $_etype714 = 0;
+            $xfer += $input->readListBegin($_etype714, $_size711);
+            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
             {
-              $elem709 = null;
-              $xfer += $input->readString($elem709);
-              $this->success []= $elem709;
+              $elem716 = null;
+              $xfer += $input->readString($elem716);
+              $this->success []= $elem716;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17442,9 +17442,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter710)
+          foreach ($this->success as $iter717)
           {
-            $xfer += $output->writeString($iter710);
+            $xfer += $output->writeString($iter717);
           }
         }
         $output->writeListEnd();
@@ -18757,15 +18757,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size711 = 0;
-            $_etype714 = 0;
-            $xfer += $input->readListBegin($_etype714, $_size711);
-            for ($_i715 = 0; $_i715 < $_size711; ++$_i715)
+            $_size718 = 0;
+            $_etype721 = 0;
+            $xfer += $input->readListBegin($_etype721, $_size718);
+            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
             {
-              $elem716 = null;
-              $elem716 = new \metastore\Partition();
-              $xfer += $elem716->read($input);
-              $this->new_parts []= $elem716;
+              $elem723 = null;
+              $elem723 = new \metastore\Partition();
+              $xfer += $elem723->read($input);
+              $this->new_parts []= $elem723;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18793,9 +18793,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter717)
+          foreach ($this->new_parts as $iter724)
           {
-            $xfer += $iter717->write($output);
+            $xfer += $iter724->write($output);
           }
         }
         $output->writeListEnd();
@@ -19010,15 +19010,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size718 = 0;
-            $_etype721 = 0;
-            $xfer += $input->readListBegin($_etype721, $_size718);
-            for ($_i722 = 0; $_i722 < $_size718; ++$_i722)
+            $_size725 = 0;
+            $_etype728 = 0;
+            $xfer += $input->readListBegin($_etype728, $_size725);
+            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
             {
-              $elem723 = null;
-              $elem723 = new \metastore\PartitionSpec();
-              $xfer += $elem723->read($input);
-              $this->new_parts []= $elem723;
+              $elem730 = null;
+              $elem730 = new \metastore\PartitionSpec();
+              $xfer += $elem730->read($input);
+              $this->new_parts []= $elem730;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19046,9 +19046,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter724)
+          foreach ($this->new_parts as $iter731)
           {
-            $xfer += $iter724->write($output);
+            $xfer += $iter731->write($output);
           }
         }
         $output->writeListEnd();
@@ -19298,14 +19298,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size725 = 0;
-            $_etype728 = 0;
-            $xfer += $input->readListBegin($_etype728, $_size725);
-            for ($_i729 = 0; $_i729 < $_size725; ++$_i729)
+            $_size732 = 0;
+            $_etype735 = 0;
+            $xfer += $input->readListBegin($_etype735, $_size732);
+            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
             {
-              $elem730 = null;
-              $xfer += $input->readString($elem730);
-              $this->part_vals []= $elem730;
+              $elem737 = null;
+              $xfer += $input->readString($elem737);
+              $this->part_vals []= $elem737;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19343,9 +19343,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter731)
+          foreach ($this->part_vals as $iter738)
           {
-            $xfer += $output->writeString($iter731);
+            $xfer += $output->writeString($iter738);
           }
         }
         $output->writeListEnd();
@@ -19847,14 +19847,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size732 = 0;
-            $_etype735 = 0;
-            $xfer += $input->readListBegin($_etype735, $_size732);
-            for ($_i736 = 0; $_i736 < $_size732; ++$_i736)
+            $_size739 = 0;
+            $_etype742 = 0;
+            $xfer += $input->readListBegin($_etype742, $_size739);
+            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
             {
-              $elem737 = null;
-              $xfer += $input->readString($elem737);
-              $this->part_vals []= $elem737;
+              $elem744 = null;
+              $xfer += $input->readString($elem744);
+              $this->part_vals []= $elem744;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19900,9 +19900,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter738)
+          foreach ($this->part_vals as $iter745)
           {
-            $xfer += $output->writeString($iter738);
+            $xfer += $output->writeString($iter745);
           }
         }
         $output->writeListEnd();
@@ -20756,14 +20756,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size739 = 0;
-            $_etype742 = 0;
-            $xfer += $input->readListBegin($_etype742, $_size739);
-            for ($_i743 = 0; $_i743 < $_size739; ++$_i743)
+            $_size746 = 0;
+            $_etype749 = 0;
+            $xfer += $input->readListBegin($_etype749, $_size746);
+            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
             {
-              $elem744 = null;
-              $xfer += $input->readString($elem744);
-              $this->part_vals []= $elem744;
+              $elem751 = null;
+              $xfer += $input->readString($elem751);
+              $this->part_vals []= $elem751;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20808,9 +20808,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter745)
+          foreach ($this->part_vals as $iter752)
           {
-            $xfer += $output->writeString($iter745);
+            $xfer += $output->writeString($iter752);
           }
         }
         $output->writeListEnd();
@@ -21063,14 +21063,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size746 = 0;
-            $_etype749 = 0;
-            $xfer += $input->readListBegin($_etype749, $_size746);
-            for ($_i750 = 0; $_i750 < $_size746; ++$_i750)
+            $_size753 = 0;
+            $_etype756 = 0;
+            $xfer += $input->readListBegin($_etype756, $_size753);
+            for ($_i757 = 0; $_i757 < $_size753; ++$_i757)
             {
-              $elem751 = null;
-              $xfer += $input->readString($elem751);
-              $this->part_vals []= $elem751;
+              $elem758 = null;
+              $xfer += $input->readString($elem758);
+              $this->part_vals []= $elem758;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21123,9 +21123,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter752)
+          foreach ($this->part_vals as $iter759)
           {
-            $xfer += $output->writeString($iter752);
+            $xfer += $output->writeString($iter759);
           }
         }
         $output->writeListEnd();
@@ -22139,14 +22139,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size753 = 0;
-            $_etype756 = 0;
-            $xfer += $input->readListBegin($_etype756, $_size753);
-            for ($_i757 = 0; $_i757 < $_size753; ++$_i757)
+            $_size760 = 0;
+            $_etype763 = 0;
+            $xfer += $input->readListBegin($_etype763, $_size760);
+            for ($_i764 = 0; $_i764 < $_size760; ++$_i764)
             {
-              $elem758 = null;
-              $xfer += $input->readString($elem758);
-              $this->part_vals []= $elem758;
+              $elem765 = null;
+              $xfer += $input->readString($elem765);
+              $this->part_vals []= $elem765;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22184,9 +22184,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter759)
+          foreach ($this->part_vals as $iter766)
           {
-            $xfer += $output->writeString($iter759);
+            $xfer += $output->writeString($iter766);
           }
         }
         $output->writeListEnd();
@@ -22428,17 +22428,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size760 = 0;
-            $_ktype761 = 0;
-            $_vtype762 = 0;
-            $xfer += $input->readMapBegin($_ktype761, $_vtype762, $_size760);
-            for ($_i764 = 0; $_i764 < $_size760; ++$_i764)
+            $_size767 = 0;
+            $_ktype768 = 0;
+            $_vtype769 = 0;
+            $xfer += $input->readMapBegin($_ktype768, $_vtype769, $_size767);
+            for ($_i771 = 0; $_i771 < $_size767; ++$_i771)
             {
-              $key765 = '';
-              $val766 = '';
-              $xfer += $input->readString($key765);
-              $xfer += $input->readString($val766);
-              $this->partitionSpecs[$key765] = $val766;
+              $key772 = '';
+              $val773 = '';
+              $xfer += $input->readString($key772);
+              $xfer += $input->readString($val773);
+              $this->partitionSpecs[$key772] = $val773;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22494,10 +22494,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter767 => $viter768)
+          foreach ($this->partitionSpecs as $kiter774 => $viter775)
           {
-            $xfer += $output->writeString($kiter767);
-            $xfer += $output->writeString($viter768);
+            $xfer += $output->writeString($kiter774);
+            $xfer += $output->writeString($viter775);
           }
         }
         $output->writeMapEnd();
@@ -22809,17 +22809,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size769 = 0;
-            $_ktype770 = 0;
-            $_vtype771 = 0;
-            $xfer += $input->readMapBegin($_ktype770, $_vtype771, $_size769);
-            for ($_i773 = 0; $_i773 < $_size769; ++$_i773)
+            $_size776 = 0;
+            $_ktype777 = 0;
+            $_vtype778 = 0;
+            $xfer += $input->readMapBegin($_ktype777, $_vtype778, $_size776);
+            for ($_i780 = 0; $_i780 < $_size776; ++$_i780)
             {
-              $key774 = '';
-              $val775 = '';
-              $xfer += $input->readString($key774);
-              $xfer += $input->readString($val775);
-              $this->partitionSpecs[$key774] = $val775;
+              $key781 = '';
+              $val782 = '';
+              $xfer += $input->readString($key781);
+              $xfer += $input->readString($val782);
+              $this->partitionSpecs[$key781] = $val782;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22875,10 +22875,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter776 => $viter777)
+          foreach ($this->partitionSpecs as $kiter783 => $viter784)
           {
-            $xfer += $output->writeString($kiter776);
-            $xfer += $output->writeString($viter777);
+            $xfer += $output->writeString($kiter783);
+            $xfer += $output->writeString($viter784);
           }
         }
         $output->writeMapEnd();
@@ -23011,15 +23011,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
             {
-              $elem783 = null;
-              $elem783 = new \metastore\Partition();
-              $xfer += $elem783->read($input);
-              $this->success []= $elem783;
+              $elem790 = null;
+              $elem790 = new \metastore\Partition();
+              $xfer += $elem790->read($input);
+              $this->success []= $elem790;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23079,9 +23079,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter784)
+          foreach ($this->success as $iter791)
           {
-            $xfer += $iter784->write($output);
+            $xfer += $iter791->write($output);
           }
         }
         $output->writeListEnd();
@@ -23227,14 +23227,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size792 = 0;
+            $_etype795 = 0;
+            $xfer += $input->readListBegin($_etype795, $_size792);
+            for ($_i796 = 0; $_i796 < $_size792; ++$_i796)
             {
-              $elem790 = null;
-              $xfer += $input->readString($elem790);
-              $this->part_vals []= $elem790;
+              $elem797 = null;
+              $xfer += $input->readString($elem797);
+              $this->part_vals []= $elem797;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23251,14 +23251,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size791 = 0;
-            $_etype794 = 0;
-            $xfer += $input->readListBegin($_etype794, $_size791);
-            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
+            $_size798 = 0;
+            $_etype801 = 0;
+            $xfer += $input->readListBegin($_etype801, $_size798);
+            for ($_i802 = 0; $_i802 < $_size798; ++$_i802)
             {
-              $elem796 = null;
-              $xfer += $input->readString($elem796);
-              $this->group_names []= $elem796;
+              $elem803 = null;
+              $xfer += $input->readString($elem803);
+              $this->group_names []= $elem803;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23296,9 +23296,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter797)
+          foreach ($this->part_vals as $iter804)
           {
-            $xfer += $output->writeString($iter797);
+            $xfer += $output->writeString($iter804);
           }
         }
         $output->writeListEnd();
@@ -23318,9 +23318,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter798)
+          foreach ($this->group_names as $iter805)
           {
-            $xfer += $output->writeString($iter798);
+            $xfer += $output->writeString($iter805);
           }
         }
         $output->writeListEnd();
@@ -23911,15 +23911,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem804 = null;
-              $elem804 = new \metastore\Partition();
-              $xfer += $elem804->read($input);
-              $this->success []= $elem804;
+              $elem811 = null;
+              $elem811 = new \metastore\Partition();
+              $xfer += $elem811->read($input);
+              $this->success []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23963,9 +23963,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter805)
+          foreach ($this->success as $iter812)
           {
-            $xfer += $iter805->write($output);
+            $xfer += $iter812->write($output);
           }
         }
         $output->writeListEnd();
@@ -24111,14 +24111,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size813 = 0;
+            $_etype816 = 0;
+            $xfer += $input->readListBegin($_etype816, $_size813);
+            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
             {
-              $elem811 = null;
-              $xfer += $input->readString($elem811);
-              $this->group_names []= $elem811;
+              $elem818 = null;
+              $xfer += $input->readString($elem818);
+              $this->group_names []= $elem818;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24166,9 +24166,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter812)
+          foreach ($this->group_names as $iter819)
           {
-            $xfer += $output->writeString($iter812);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -24257,15 +24257,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\Partition();
-              $xfer += $elem818->read($input);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $elem825 = new \metastore\Partition();
+              $xfer += $elem825->read($input);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24309,9 +24309,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter819)
+          foreach ($this->success as $iter826)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -24531,15 +24531,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem825 = null;
-              $elem825 = new \metastore\PartitionSpec();
-              $xfer += $elem825->read($input);
-              $this->success []= $elem825;
+              $elem832 = null;
+              $elem832 = new \metastore\PartitionSpec();
+              $xfer += $elem832->read($input);
+              $this->success []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24583,9 +24583,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter826)
+          foreach ($this->success as $iter833)
           {
-            $xfer += $iter826->write($output);
+            $xfer += $iter833->write($output);
           }
         }
         $output->writeListEnd();
@@ -24792,14 +24792,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size827 = 0;
-            $_etype830 = 0;
-            $xfer += $input->readListBegin($_etype830, $_size827);
-            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
+            $_size834 = 0;
+            $_etype837 = 0;
+            $xfer += $input->readListBegin($_etype837, $_size834);
+            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
             {
-              $elem832 = null;
-              $xfer += $input->readString($elem832);
-              $this->success []= $elem832;
+              $elem839 = null;
+              $xfer += $input->readString($elem839);
+              $this->success []= $elem839;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24835,9 +24835,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter833)
+          foreach ($this->success as $iter840)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter840);
           }
         }
         $output->writeListEnd();
@@ -24953,14 +24953,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem839 = null;
-              $xfer += $input->readString($elem839);
-              $this->part_vals []= $elem839;
+              $elem846 = null;
+              $xfer += $input->readString($elem846);
+              $this->part_vals []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25005,9 +25005,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter840)
+          foreach ($this->part_vals as $iter847)
           {
-            $xfer += $output->writeString($iter840);
+            $xfer += $output->writeString($iter847);
           }
         }
         $output->writeListEnd();
@@ -25101,15 +25101,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size841 = 0;
-            $_etype844 = 0;
-            $xfer += $input->readListBegin($_etype844, $_size841);
-            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem846 = null;
-              $elem846 = new \metastore\Partition();
-              $xfer += $elem846->read($input);
-              $this->success []= $elem846;
+              $elem853 = null;
+              $elem853 = new \metastore\Partition();
+              $xfer += $elem853->read($input);
+              $this->success []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25153,9 +25153,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter847)
+          foreach ($this->success as $iter854)
           {
-            $xfer += $iter847->write($output);
+            $xfer += $iter854->write($output);
           }
         }
         $output->writeListEnd();
@@ -25302,14 +25302,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size848 = 0;
-            $_etype851 = 0;
-            $xfer += $input->readListBegin($_etype851, $_size848);
-            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
+            $_size855 = 0;
+            $_etype858 = 0;
+            $xfer += $input->readListBegin($_etype858, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $elem853 = null;
-              $xfer += $input->readString($elem853);
-              $this->part_vals []= $elem853;
+              $elem860 = null;
+              $xfer += $input->readString($elem860);
+              $this->part_vals []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25333,14 +25333,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size854 = 0;
-            $_etype857 = 0;
-            $xfer += $input->readListBegin($_etype857, $_size854);
-            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
+            $_size861 = 0;
+            $_etype864 = 0;
+            $xfer += $input->readListBegin($_etype864, $_size861);
+            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
             {
-              $elem859 = null;
-              $xfer += $input->readString($elem859);
-              $this->group_names []= $elem859;
+              $elem866 = null;
+              $xfer += $input->readString($elem866);
+              $this->group_names []= $elem866;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25378,9 +25378,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter860)
+          foreach ($this->part_vals as $iter867)
           {
-            $xfer += $output->writeString($iter860);
+            $xfer += $output->writeString($iter867);
           }
         }
         $output->writeListEnd();
@@ -25405,9 +25405,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter861)
+          foreach ($this->group_names as $iter868)
           {
-            $xfer += $output->writeString($iter861);
+            $xfer += $output->writeString($iter868);
           }
         }
         $output->writeListEnd();
@@ -25496,15 +25496,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size862 = 0;
-            $_etype865 = 0;
-            $xfer += $input->readListBegin($_etype865, $_size862);
-            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
+            $_size869 = 0;
+            $_etype872 = 0;
+            $xfer += $input->readListBegin($_etype872, $_size869);
+            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
             {
-              $elem867 = null;
-              $elem867 = new \metastore\Partition();
-              $xfer += $elem867->read($input);
-              $this->success []= $elem867;
+              $elem874 = null;
+              $elem874 = new \metastore\Partition();
+              $xfer += $elem874->read($input);
+              $this->success []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25548,9 +25548,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter868)
+          foreach ($this->success as $iter875)
           {
-            $xfer += $iter868->write($output);
+            $xfer += $iter875->write($output);
           }
         }
         $output->writeListEnd();
@@ -25671,14 +25671,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size869 = 0;
-            $_etype872 = 0;
-            $xfer += $input->readListBegin($_etype872, $_size869);
-            for ($_i873 = 0; $_i873 < $_size869; ++$_i873)
+            $_size876 = 0;
+            $_etype879 = 0;
+            $xfer += $input->readListBegin($_etype879, $_size876);
+            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
             {
-              $elem874 = null;
-              $xfer += $input->readString($elem874);
-              $this->part_vals []= $elem874;
+              $elem881 = null;
+              $xfer += $input->readString($elem881);
+              $this->part_vals []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25723,9 +25723,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter875)
+          foreach ($this->part_vals as $iter882)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter882);
           }
         }
         $output->writeListEnd();
@@ -25818,14 +25818,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size876 = 0;
-            $_etype879 = 0;
-            $xfer += $input->readListBegin($_etype879, $_size876);
-            for ($_i880 = 0; $_i880 < $_size876; ++$_i880)
+            $_size883 = 0;
+            $_etype886 = 0;
+            $xfer += $input->readListBegin($_etype886, $_size883);
+            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
             {
-              $elem881 = null;
-              $xfer += $input->readString($elem881);
-              $this->success []= $elem881;
+              $elem888 = null;
+              $xfer += $input->readString($elem888);
+              $this->success []= $elem888;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25869,9 +25869,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter882)
+          foreach ($this->success as $iter889)
           {
-            $xfer += $output->writeString($iter882);
+            $xfer += $output->writeString($iter889);
           }
         }
         $output->writeListEnd();
@@ -26114,15 +26114,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size890 = 0;
+            $_etype893 = 0;
+            $xfer += $input->readListBegin($_etype893, $_size890);
+            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\Partition();
-              $xfer += $elem888->read($input);
-              $this->success []= $elem888;
+              $elem895 = null;
+              $elem895 = new \metastore\Partition();
+              $xfer += $elem895->read($input);
+              $this->success []= $elem895;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26166,9 +26166,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter889)
+          foreach ($this->success as $iter896)
           {
-            $xfer += $iter889->write($output);
+            $xfer += $iter896->write($output);
           }
         }
         $output->writeListEnd();
@@ -26411,15 +26411,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size890 = 0;
-            $_etype893 = 0;
-            $xfer += $input->readListBegin($_etype893, $_size890);
-            for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
+            $_size897 = 0;
+            $_etype900 = 0;
+            $xfer += $input->readListBegin($_etype900, $_size897);
+            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
             {
-              $elem895 = null;
-              $elem895 = new \metastore\PartitionSpec();
-              $xfer += $elem895->read($input);
-              $this->success []= $elem895;
+              $elem902 = null;
+              $elem902 = new \metastore\PartitionSpec();
+              $xfer += $elem902->read($input);
+              $this->success []= $elem902;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26463,9 +26463,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter896)
+          foreach ($this->success as $iter903)
           {
-            $xfer += $iter896->write($output);
+            $xfer += $iter903->write($output);
           }
         }
         $output->writeListEnd();
@@ -27031,14 +27031,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size897 = 0;
-            $_etype900 = 0;
-            $xfer += $input->readListBegin($_etype900, $_size897);
-            for ($_i901 = 0; $_i901 < $_size897; ++$_i901)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem902 = null;
-              $xfer += $input->readString($elem902);
-              $this->names []= $elem902;
+              $elem909 = null;
+              $xfer += $input->readString($elem909);
+              $this->names []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27076,9 +27076,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter903)
+          foreach ($this->names as $iter910)
           {
-            $xfer += $output->writeString($iter903);
+            $xfer += $output->writeString($iter910);
           }
         }
         $output->writeListEnd();
@@ -27167,15 +27167,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size904 = 0;
-            $_etype907 = 0;
-            $xfer += $input->readListBegin($_etype907, $_size904);
-            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
+            $_size911 = 0;
+            $_etype914 = 0;
+            $xfer += $input->readListBegin($_etype914, $_size911);
+            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
             {
-              $elem909 = null;
-              $elem909 = new \metastore\Partition();
-              $xfer += $elem909->read($input);
-              $this->success []= $elem909;
+              $elem916 = null;
+              $elem916 = new \metastore\Partition();
+              $xfer += $elem916->read($input);
+              $this->success []= $elem916;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27219,9 +27219,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter910)
+          foreach ($this->success as $iter917)
           {
-            $xfer += $iter910->write($output);
+            $xfer += $iter917->write($output);
           }
         }
         $output->writeListEnd();
@@ -27560,15 +27560,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size918 = 0;
+            $_etype921 = 0;
+            $xfer += $input->readListBegin($_etype921, $_size918);
+            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
             {
-              $elem916 = null;
-              $elem916 = new \metastore\Partition();
-              $xfer += $elem916->read($input);
-              $this->new_parts []= $elem916;
+              $elem923 = null;
+              $elem923 = new \metastore\Partition();
+              $xfer += $elem923->read($input);
+              $this->new_parts []= $elem923;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27606,9 +27606,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter917)
+          foreach ($this->new_parts as $iter924)
           {
-            $xfer += $iter917->write($output);
+            $xfer += $iter924->write($output);
           }
         }
         $output->writeListEnd();
@@ -27823,15 +27823,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size918 = 0;
-            $_etype921 = 0;
-            $xfer += $input->readListBegin($_etype921, $_size918);
-            for ($_i922 = 0; $_i922 < $_size918; ++$_i922)
+            $_size925 = 0;
+            $_etype928 = 0;
+            $xfer += $input->readListBegin($_etype928, $_size925);
+            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
             {
-              $elem923 = null;
-              $elem923 = new \metastore\Partition();
-              $xfer += $elem923->read($input);
-              $this->new_parts []= $elem923;
+              $elem930 = null;
+              $elem930 = new \metastore\Partition();
+              $xfer += $elem930->read($input);
+              $this->new_parts []= $elem930;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27877,9 +27877,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter924)
+          foreach ($this->new_parts as $iter931)
           {
-            $xfer += $iter924->write($output);
+            $xfer += $iter931->write($output);
           }
         }
         $output->writeListEnd();
@@ -28357,14 +28357,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size925 = 0;
-            $_etype928 = 0;
-            $xfer += $input->readListBegin($_etype928, $_size925);
-            for ($_i929 = 0; $_i929 < $_size925; ++$_i929)
+            $_size932 = 0;
+            $_etype935 = 0;
+            $xfer += $input->readListBegin($_etype935, $_size932);
+            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
             {
-              $elem930 = null;
-              $xfer += $input->readString($elem930);
-              $this->part_vals []= $elem930;
+              $elem937 = null;
+              $xfer += $input->readString($elem937);
+              $this->part_vals []= $elem937;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28410,9 +28410,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter931)
+          foreach ($this->part_vals as $iter938)
           {
-            $xfer += $output->writeString($iter931);
+            $xfer += $output->writeString($iter938);
           }
         }
         $output->writeListEnd();
@@ -28597,14 +28597,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size932 = 0;
-            $_etype935 = 0;
-            $xfer += $input->readListBegin($_etype935, $_size932);
-            for ($_i936 = 0; $_i936 < $_size932; ++$_i936)
+            $_size939 = 0;
+            $_etype942 = 0;
+            $xfer += $input->readListBegin($_etype942, $_size939);
+            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
             {
-              $elem937 = null;
-              $xfer += $input->readString($elem937);
-              $this->part_vals []= $elem937;
+              $elem944 = null;
+              $xfer += $input->readString($elem944);
+              $this->part_vals []= $elem944;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28639,9 +28639,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter938)
+          foreach ($this->part_vals as $iter945)
           {
-            $xfer += $output->writeString($iter938);
+            $xfer += $output->writeString($iter945);
           }
         }
         $output->writeListEnd();
@@ -29095,14 +29095,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size939 = 0;
-            $_etype942 = 0;
-            $xfer += $input->readListBegin($_etype942, $_size939);
-            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
+            $_size946 = 0;
+            $_etype949 = 0;
+            $xfer += $input->readListBegin($_etype949, $_size946);
+            for ($_i950 = 0; $_i950 < $_size946; ++$_i950)
             {
-              $elem944 = null;
-              $xfer += $input->readString($elem944);
-              $this->success []= $elem944;
+              $elem951 = null;
+              $xfer += $input->readString($elem951);
+              $this->success []= $elem951;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29138,9 +29138,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter945)
+          foreach ($this->success as $iter952)
           {
-            $xfer += $output->writeString($iter945);
+            $xfer += $output->writeString($iter952);
           }
         }
         $output->writeListEnd();
@@ -29300,17 +29300,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size946 = 0;
-            $_ktype947 = 0;
-            $_vtype948 = 0;
-            $xfer += $input->readMapBegin($_ktype947, $_vtype948, $_size946);
-            for ($_i950 = 0; $_i950 < $_size946; ++$_i950)
+            $_size953 = 0;
+            $_ktype954 = 0;
+            $_vtype955 = 0;
+            $xfer += $input->readMapBegin($_ktype954, $_vtype955, $_size953);
+            for ($_i957 = 0; $_i957 < $_size953; ++$_i957)
             {
-              $key951 = '';
-              $val952 = '';
-              $xfer += $input->readString($key951);
-              $xfer += $input->readString($val952);
-              $this->success[$key951] = $val952;
+              $key958 = '';
+              $val959 = '';
+              $xfer += $input->readString($key958);
+              $xfer += $input->readString($val959);
+              $this->success[$key958] = $val959;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29346,10 +29346,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter953 => $viter954)
+          foreach ($this->success as $kiter960 => $viter961)
           {
-            $xfer += $output->writeString($kiter953);
-            $xfer += $output->writeString($viter954);
+            $xfer += $output->writeString($kiter960);
+            $xfer += $output->writeString($viter961);
           }
         }
         $output->writeMapEnd();
@@ -29469,17 +29469,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size955 = 0;
-            $_ktype956 = 0;
-            $_vtype957 = 0;
-            $xfer += $input->readMapBegin($_ktype956, $_vtype957, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size962 = 0;
+            $_ktype963 = 0;
+            $_vtype964 = 0;
+            $xfer += $input->readMapBegin($_ktype963, $_vtype964, $_size962);
+            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
             {
-              $key960 = '';
-              $val961 = '';
-              $xfer += $input->readString($key960);
-              $xfer += $input->readString($val961);
-              $this->part_vals[$key960] = $val961;
+              $key967 = '';
+              $val968 = '';
+              $xfer += $input->readString($key967);
+              $xfer += $input->readString($val968);
+              $this->part_vals[$key967] = $val968;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29524,10 +29524,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter962 => $viter963)
+          foreach ($this->part_vals as $kiter969 => $viter970)
           {
-            $xfer += $output->writeString($kiter962);
-            $xfer += $output->writeString($viter963);
+            $xfer += $output->writeString($kiter969);
+            $xfer += $output->writeString($viter970);
           }
         }
         $output->writeMapEnd();
@@ -29849,17 +29849,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size964 = 0;
-            $_ktype965 = 0;
-            $_vtype966 = 0;
-            $xfer += $input->readMapBegin($_ktype965, $_vtype966, $_size964);
-            for ($_i968 = 0; $_i968 < $_size964; ++$_i968)
+            $_size971 = 0;
+            $_ktype972 = 0;
+            $_vtype973 = 0;
+            $xfer += $input->readMapBegin($_ktype972, $_vtype973, $_size971);
+            for ($_i975 = 0; $_i975 < $_size971; ++$_i975)
             {
-              $key969 = '';
-              $val970 = '';
-              $xfer += $input->readString($key969);
-              $xfer += $input->readString($val970);
-              $this->part_vals[$key969] = $val970;
+              $key976 = '';
+              $val977 = '';
+              $xfer += $input->readString($key976);
+              $xfer += $input->readString($val977);
+              $this->part_vals[$key976] = $val977;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29904,10 +29904,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter971 => $viter972)
+          foreach ($this->part_vals as $kiter978 => $viter979)
           {
-            $xfer += $output->writeString($kiter971);
-            $xfer += $output->writeString($viter972);
+            $xfer += $output->writeString($kiter978);
+            $xfer += $output->writeString($viter979);
           }
         }
         $output->writeMapEnd();
@@ -31381,15 +31381,15 @@ class ThriftHiveMetastore_get_indexes_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size973 = 0;
-            $_etype976 = 0;
-            $xfer += $input->readListBegin($_etype976, $_size973);
-            for ($_i977 = 0; $_i977 < $_size973; ++$_i977)
+            $_size980 = 0;
+            $_etype983 = 0;
+            $xfer += $input->readListBegin($_etype983, $_size980);
+            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
             {
-              $elem978 = null;
-              $elem978 = new \metastore\Index();
-              $xfer += $elem978->read($input);
-              $this->success []= $elem978;
+              $elem985 = null;
+              $elem985 = new \metastore\Index();
+              $xfer += $elem985->read($input);
+              $this->success []= $elem985;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31433,9 +31433,9 @@ class ThriftHiveMetastore_get_indexes_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter979)
+          foreach ($this->success as $iter986)
           {
-            $xfer += $iter979->write($output);
+            $xfer += $iter986->write($output);
           }
         }
         $output->writeListEnd();
@@ -31642,14 +31642,14 @@ class ThriftHiveMetastore_get_index_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size980 = 0;
-            $_etype983 = 0;
-            $xfer += $input->readListBegin($_etype983, $_size980);
-            for ($_i984 = 0; $_i984 < $_size980; ++$_i984)
+            $_size987 = 0;
+            $_etype990 = 0;
+            $xfer += $input->readListBegin($_etype990, $_size987);
+            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
             {
-              $elem985 = null;
-              $xfer += $input->readString($elem985);
-              $this->success []= $elem985;
+              $elem992 = null;
+              $xfer += $input->readString($elem992);
+              $this->success []= $elem992;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31685,9 +31685,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter986)
+          foreach ($this->success as $iter993)
           {
-            $xfer += $output->writeString($iter986);
+            $xfer += $output->writeString($iter993);
           }
         }
         $output->writeListEnd();
@@ -35581,14 +35581,14 @@ class ThriftHiveMetastore_get_functions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size987 = 0;
-            $_etype990 = 0;
-            $xfer += $input->readListBegin($_etype990, $_size987);
-            for ($_i991 = 0; $_i991 < $_size987; ++$_i991)
+            $_size994 = 0;
+            $_etype997 = 0;
+            $xfer += $input->readListBegin($_etype997, $_size994);
+            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
             {
-              $elem992 = null;
-              $xfer += $input->readString($elem992);
-              $this->success []= $elem992;
+              $elem999 = null;
+              $xfer += $input->readString($elem999);
+              $this->success []= $elem999;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35624,9 +35624,9 @@ class ThriftHiveMetastore_get_functions_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter993)
+          foreach ($this->success as $iter1000)
           {
-            $xfer += $output->writeString($iter993);
+            $xfer += $output->writeString($iter1000);
           }
         }
         $output->writeListEnd();
@@ -36495,14 +36495,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size994 = 0;
-            $_etype997 = 0;
-            $xfer += $input->readListBegin($_etype997, $_size994);
-            for ($_i998 = 0; $_i998 < $_size994; ++$_i998)
+            $_size1001 = 0;
+            $_etype1004 = 0;
+            $xfer += $input->readListBegin($_etype1004, $_size1001);
+            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
             {
-              $elem999 = null;
-              $xfer += $input->readString($elem999);
-              $this->success []= $elem999;
+              $elem1006 = null;
+              $xfer += $input->readString($elem1006);
+              $this->success []= $elem1006;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36538,9 +36538,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1000)
+          foreach ($this->success as $iter1007)
           {
-            $xfer += $output->writeString($iter1000);
+            $xfer += $output->writeString($iter1007);
           }
         }
         $output->writeListEnd();
@@ -37231,15 +37231,15 @@ class ThriftHiveMetastore_list_roles_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1001 = 0;
-            $_etype1004 = 0;
-            $xfer += $input->readListBegin($_etype1004, $_size1001);
-            for ($_i1005 = 0; $_i1005 < $_size1001; ++$_i1005)
+            $_size1008 = 0;
+            $_etype1011 = 0;
+            $xfer += $input->readListBegin($_etype1011, $_size1008);
+            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
             {
-              $elem1006 = null;
-              $elem1006 = new \metastore\Role();
-              $xfer += $elem1006->read($input);
-              $this->success []= $elem1006;
+              $elem1013 = null;
+              $elem1013 = new \metastore\Role();
+              $xfer += $elem1013->read($input);
+              $this->success []= $elem1013;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37275,9 +37275,9 @@ class ThriftHiveMetastore_list_roles_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1007)
+          foreach ($this->success as $iter1014)
           {
-            $xfer += $iter1007->write($output);
+            $xfer += $iter1014->write($output);
           }
         }
         $output->writeListEnd();
@@ -37939,14 +37939,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1008 = 0;
-            $_etype1011 = 0;
-            $xfer += $input->readListBegin($_etype1011, $_size1008);
-            for ($_i1012 = 0; $_i1012 < $_size1008; ++$_i1012)
+            $_size1015 = 0;
+            $_etype1018 = 0;
+            $xfer += $input->readListBegin($_etype1018, $_size1015);
+            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
             {
-              $elem1013 = null;
-              $xfer += $input->readString($elem1013);
-              $this->group_names []= $elem1013;
+              $elem1020 = null;
+              $xfer += $input->readString($elem1020);
+              $this->group_names []= $elem1020;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -37987,9 +37987,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1014)
+          foreach ($this->group_names as $iter1021)
           {
-            $xfer += $output->writeString($iter1014);
+            $xfer += $output->writeString($iter1021);
           }
         }
         $output->writeListEnd();
@@ -38297,15 +38297,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1015 = 0;
-            $_etype1018 = 0;
-            $xfer += $input->readListBegin($_etype1018, $_size1015);
-            for ($_i1019 = 0; $_i1019 < $_size1015; ++$_i1019)
+            $_size1022 = 0;
+            $_etype1025 = 0;
+            $xfer += $input->readListBegin($_etype1025, $_size1022);
+            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
             {
-              $elem1020 = null;
-              $elem1020 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem1020->read($input);
-              $this->success []= $elem1020;
+              $elem1027 = null;
+              $elem1027 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem1027->read($input);
+              $this->success []= $elem1027;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -38341,9 +38341,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1021)
+          foreach ($this->success as $iter1028)
           {
-            $xfer += $iter1021->write($output);
+            $xfer += $iter1028->write($output);
           }
         }
         $output->writeListEnd();
@@ -38975,14 +38975,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1022 = 0;
-            $_etype1025 = 0;
-            $xfer += $input->readListBegin($_etype1025, $_size1022);
-            for ($_i1026 = 0; $_i1026 < $_size1022; ++$_i1026)
+            $_size1029 = 0;
+            $_etype1032 = 0;
+            $xfer += $input->readListBegin($_etype1032, $_size1029);
+            for ($_i1033 = 0; $_i1033 < $_size1029; ++$_i1033)
             {
-              $elem1027 = null;
-              $xfer += $input->readString($elem1027);
-              $this->group_names []= $elem1027;
+              $elem1034 = null;
+              $xfer += $input->readString($elem1034);
+              $this->group_names []= $elem1034;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -39015,9 +39015,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1028)
+          foreach ($this->group_names as $iter1035)
           {
-            $xfer += $output->writeString($iter1028);
+            $xfer += $output->writeString($iter1035);
           }
         }
         $output->writeListEnd();
@@ -39093,14 +39093,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1029 = 0;
-            $_etype1032 = 0;
-            $xfer += $input->readListBegin($_etype1032, $_size1029);
-            for ($_i1033 = 0; $_i1033 < $_size1029; ++$_i1033)
+            $_size1036 = 0;
+            $_etype1039 = 0;
+            $xfer += $input->readListBegin($_etype1039, $_size1036);
+            for ($_i1040 = 0; $_i1040 < $_size1036; ++$_i1040)
             {
-              $elem1034 = null;
-              $xfer += $input->readString($elem1034);
-              $this->success []= $elem1034;
+              $elem1041 = null;
+              $xfer += $input->readString($elem1041);
+              $this->success []= $elem1041;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -39136,9 +39136,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1035)
+          foreach ($this->success as $iter1042)
           {
-            $xfer += $output->writeString($iter1035);
+            $xfer += $output->writeString($iter1042);
           }
         }
         $output->writeListEnd();
@@ -40255,14 +40255,14 @@ class ThriftHiveMetastore_get_all_token_identifiers_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1036 = 

<TRUNCATED>

[8/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)


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

Branch: refs/heads/master
Commit: bbd99ed60e5708af3dc329b097d4b024f73041bd
Parents: 12f5550
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Mon Dec 19 14:02:01 2016 -0800
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Mon Dec 19 14:02:01 2016 -0800

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |   55 +-
 .../listener/TestDbNotificationListener.java    |  955 +++++---
 metastore/if/hive_metastore.thrift              |    4 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2054 ++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  821 ++++---
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   13 +
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../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 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/InsertEventRequestData.java   |  181 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 2220 +++++++++---------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1250 +++++-----
 .../src/gen/thrift/gen-php/metastore/Types.php  |  291 ++-
 .../hive_metastore/ThriftHiveMetastore.py       |  842 +++----
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  185 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   28 +-
 .../hive/metastore/events/InsertEvent.java      |   36 +-
 .../metastore/messaging/MessageFactory.java     |   22 +-
 .../messaging/json/JSONInsertMessage.java       |   54 +-
 .../messaging/json/JSONMessageFactory.java      |   36 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   56 +-
 .../apache/hadoop/fs/ProxyLocalFileSystem.java  |  104 +-
 31 files changed, 5202 insertions(+), 4429 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 119801f..8d29bfc 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hive.metastore.events.InsertEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
-import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -105,6 +104,7 @@ public class DbNotificationListener extends MetaStoreEventListener {
    * @param tableEvent table event.
    * @throws org.apache.hadoop.hive.metastore.api.MetaException
    */
+  @Override
   public void onConfigChange(ConfigChangeEvent tableEvent) throws MetaException {
     String key = tableEvent.getKey();
     if (key.equals(HiveConf.ConfVars.METASTORE_EVENT_DB_LISTENER_TTL.toString())) {
@@ -122,6 +122,7 @@ public class DbNotificationListener extends MetaStoreEventListener {
    * @param tableEvent table event.
    * @throws MetaException
    */
+  @Override
   public void onCreateTable(CreateTableEvent tableEvent) throws MetaException {
     Table t = tableEvent.getTable();
     NotificationEvent event =
@@ -129,13 +130,14 @@ public class DbNotificationListener extends MetaStoreEventListener {
             .buildCreateTableMessage(t).toString());
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param tableEvent table event.
    * @throws MetaException
    */
+  @Override
   public void onDropTable(DropTableEvent tableEvent) throws MetaException {
     Table t = tableEvent.getTable();
     NotificationEvent event =
@@ -143,13 +145,14 @@ public class DbNotificationListener extends MetaStoreEventListener {
             .buildDropTableMessage(t).toString());
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param tableEvent alter table event
    * @throws MetaException
    */
+  @Override
   public void onAlterTable(AlterTableEvent tableEvent) throws MetaException {
     Table before = tableEvent.getOldTable();
     Table after = tableEvent.getNewTable();
@@ -158,13 +161,14 @@ public class DbNotificationListener extends MetaStoreEventListener {
             .buildAlterTableMessage(before, after).toString());
     event.setDbName(after.getDbName());
     event.setTableName(after.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param partitionEvent partition event
    * @throws MetaException
    */
+  @Override
   public void onAddPartition(AddPartitionEvent partitionEvent) throws MetaException {
     Table t = partitionEvent.getTable();
     String msg = msgFactory
@@ -173,13 +177,14 @@ public class DbNotificationListener extends MetaStoreEventListener {
         new NotificationEvent(0, now(), EventType.ADD_PARTITION.toString(), msg);
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param partitionEvent partition event
    * @throws MetaException
    */
+  @Override
   public void onDropPartition(DropPartitionEvent partitionEvent) throws MetaException {
     Table t = partitionEvent.getTable();
     NotificationEvent event =
@@ -187,13 +192,14 @@ public class DbNotificationListener extends MetaStoreEventListener {
             .buildDropPartitionMessage(t, partitionEvent.getPartitionIterator()).toString());
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param partitionEvent partition event
    * @throws MetaException
    */
+  @Override
   public void onAlterPartition(AlterPartitionEvent partitionEvent) throws MetaException {
     Partition before = partitionEvent.getOldPartition();
     Partition after = partitionEvent.getNewPartition();
@@ -202,91 +208,98 @@ public class DbNotificationListener extends MetaStoreEventListener {
             .buildAlterPartitionMessage(partitionEvent.getTable(), before, after).toString());
     event.setDbName(before.getDbName());
     event.setTableName(before.getTableName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param dbEvent database event
    * @throws MetaException
    */
+  @Override
   public void onCreateDatabase(CreateDatabaseEvent dbEvent) throws MetaException {
     Database db = dbEvent.getDatabase();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_DATABASE.toString(), msgFactory
             .buildCreateDatabaseMessage(db).toString());
     event.setDbName(db.getName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param dbEvent database event
    * @throws MetaException
    */
+  @Override
   public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException {
     Database db = dbEvent.getDatabase();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_DATABASE.toString(), msgFactory
             .buildDropDatabaseMessage(db).toString());
     event.setDbName(db.getName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param fnEvent function event
    * @throws MetaException
    */
+  @Override
   public void onCreateFunction(CreateFunctionEvent fnEvent) throws MetaException {
     Function fn = fnEvent.getFunction();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_FUNCTION.toString(), msgFactory
             .buildCreateFunctionMessage(fn).toString());
     event.setDbName(fn.getDbName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param fnEvent function event
    * @throws MetaException
    */
+  @Override
   public void onDropFunction(DropFunctionEvent fnEvent) throws MetaException {
     Function fn = fnEvent.getFunction();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_FUNCTION.toString(), msgFactory
             .buildDropFunctionMessage(fn).toString());
     event.setDbName(fn.getDbName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param indexEvent index event
    * @throws MetaException
    */
+  @Override
   public void onAddIndex(AddIndexEvent indexEvent) throws MetaException {
     Index index = indexEvent.getIndex();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_INDEX.toString(), msgFactory
             .buildCreateIndexMessage(index).toString());
     event.setDbName(index.getDbName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param indexEvent index event
    * @throws MetaException
    */
+  @Override
   public void onDropIndex(DropIndexEvent indexEvent) throws MetaException {
     Index index = indexEvent.getIndex();
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_INDEX.toString(), msgFactory
             .buildDropIndexMessage(index).toString());
     event.setDbName(index.getDbName());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param indexEvent index event
    * @throws MetaException
    */
+  @Override
   public void onAlterIndex(AlterIndexEvent indexEvent) throws MetaException {
     Index before = indexEvent.getOldIndex();
     Index after = indexEvent.getNewIndex();
@@ -294,7 +307,7 @@ public class DbNotificationListener extends MetaStoreEventListener {
         new NotificationEvent(0, now(), EventType.ALTER_INDEX.toString(), msgFactory
             .buildAlterIndexMessage(before, after).toString());
     event.setDbName(before.getDbName());
-    enqueue(event);
+    process(event);
   }
 
   @Override
@@ -302,20 +315,20 @@ public class DbNotificationListener extends MetaStoreEventListener {
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.INSERT.toString(), msgFactory.buildInsertMessage(
             insertEvent.getDb(), insertEvent.getTable(), insertEvent.getPartitionKeyValues(),
-            insertEvent.getFiles()).toString());
+            insertEvent.getFiles(), insertEvent.getFileChecksums()).toString());
     event.setDbName(insertEvent.getDb());
     event.setTableName(insertEvent.getTable());
-    enqueue(event);
+    process(event);
   }
 
   /**
    * @param partSetDoneEvent
    * @throws MetaException
    */
+  @Override
   public void onLoadPartitionDone(LoadPartitionDoneEvent partSetDoneEvent) throws MetaException {
     // TODO, we don't support this, but we should, since users may create an empty partition and
     // then load data into it.
-
   }
 
   private int now() {
@@ -329,10 +342,12 @@ public class DbNotificationListener extends MetaStoreEventListener {
     return (int)millis;
   }
 
-  private void enqueue(NotificationEvent event) {
+  // Process this notification by adding it to metastore DB
+  private void process(NotificationEvent event) {
     if (rs != null) {
-      synchronized(NOTIFICATION_TBL_LOCK) {
-        LOG.debug("DbNotif:Enqueueing : {}:{}",event.getEventId(),event.getMessage());
+      synchronized (NOTIFICATION_TBL_LOCK) {
+        LOG.debug("DbNotificationListener: Processing : {}:{}", event.getEventId(),
+            event.getMessage());
         rs.addNotificationEvent(event);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 0b691b1..9e6af8f 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -15,7 +15,7 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
-*/
+ */
 package org.apache.hive.hcatalog.listener;
 
 import static org.junit.Assert.assertEquals;
@@ -27,7 +27,7 @@ import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -53,16 +53,11 @@ import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageFactory;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hive.hcatalog.common.HCatConstants;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
 import org.codehaus.jackson.node.ObjectNode;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -70,8 +65,13 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * Tests DbNotificationListener when used as a transactional event listener
+ * (hive.metastore.transactional.event.listeners)
+ */
 public class TestDbNotificationListener {
-  private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class
+      .getName());
   private static final int EVENTS_TTL = 30;
   private static final int CLEANUP_SLEEP_TIME = 10;
   private static Map<String, String> emptyParameters = new HashMap<String, String>();
@@ -86,12 +86,11 @@ public class TestDbNotificationListener {
     HiveConf conf = new HiveConf();
     conf.setVar(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS,
         DbNotificationListener.class.getName());
-    conf.setVar(HiveConf.ConfVars.METASTORE_EVENT_DB_LISTENER_TTL, String.valueOf(EVENTS_TTL)+"s");
+    conf.setVar(HiveConf.ConfVars.METASTORE_EVENT_DB_LISTENER_TTL, String.valueOf(EVENTS_TTL) + "s");
     conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     conf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true);
     conf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
-    conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
-        DummyRawStoreFailEvent.class.getName());
+    conf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL, DummyRawStoreFailEvent.class.getName());
     Class dbNotificationListener =
         Class.forName("org.apache.hive.hcatalog.listener.DbNotificationListener");
     Class[] classes = dbNotificationListener.getDeclaredClasses();
@@ -102,8 +101,7 @@ public class TestDbNotificationListener {
         sleepTimeField.set(null, CLEANUP_SLEEP_TIME * 1000);
       }
     }
-    conf
-    .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
     SessionState.start(new CliSessionState(conf));
     msClient = new HiveMetaStoreClient(conf);
@@ -114,363 +112,488 @@ public class TestDbNotificationListener {
   public void setup() throws Exception {
     long now = System.currentTimeMillis() / 1000;
     startTime = 0;
-    if (now > Integer.MAX_VALUE) fail("Bummer, time has fallen over the edge");
-    else startTime = (int) now;
+    if (now > Integer.MAX_VALUE) {
+      fail("Bummer, time has fallen over the edge");
+    } else {
+      startTime = (int) now;
+    }
     firstEventId = msClient.getCurrentNotificationEventId().getEventId();
     DummyRawStoreFailEvent.setEventSucceed(true);
   }
 
   @Test
   public void createDatabase() throws Exception {
-    Database db = new Database("mydb", "no description", "file:/tmp", emptyParameters);
+    String dbName = "createdb";
+    String dbName2 = "createdb2";
+    String dbLocationUri = "file:/tmp";
+    String dbDescription = "no description";
+    Database db = new Database(dbName, dbDescription, dbLocationUri, emptyParameters);
     msClient.createDatabase(db);
 
+    // Read notification from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
 
+    // Read event from notification
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_CREATE_DATABASE_EVENT, event.getEventType());
-    assertEquals("mydb", event.getDbName());
+    assertEquals(EventType.CREATE_DATABASE.toString(), event.getEventType());
+    assertEquals(dbName, event.getDbName());
     assertNull(event.getTableName());
-    assertTrue(event.getMessage().matches("\\{\"eventType\":\"CREATE_DATABASE\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"mydb\",\"timestamp\":[0-9]+}"));
 
+    // Parse the message field
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    assertEquals(EventType.CREATE_DATABASE.toString(), jsonTree.get("eventType").asText());
+    assertEquals(dbName, jsonTree.get("db").asText());
+
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
-    db = new Database("mydb2", "no description", "file:/tmp", emptyParameters);
+    db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters);
     try {
       msClient.createDatabase(db);
+      fail("Error: create database should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
   }
 
   @Test
   public void dropDatabase() throws Exception {
-    Database db = new Database("dropdb", "no description", "file:/tmp", emptyParameters);
+    String dbName = "dropdb";
+    String dbName2 = "dropdb2";
+    String dbLocationUri = "file:/tmp";
+    String dbDescription = "no description";
+    Database db = new Database(dbName, dbDescription, dbLocationUri, emptyParameters);
     msClient.createDatabase(db);
-    msClient.dropDatabase("dropdb");
+    msClient.dropDatabase(dbName);
 
+    // Read notification from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+
+    // Two events: one for create db and other for drop db
     assertEquals(2, rsp.getEventsSize());
 
+    // Read event from notification
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_DROP_DATABASE_EVENT, event.getEventType());
-    assertEquals("dropdb", event.getDbName());
+    assertEquals(EventType.DROP_DATABASE.toString(), event.getEventType());
+    assertEquals(dbName, event.getDbName());
     assertNull(event.getTableName());
-    assertTrue(event.getMessage().matches("\\{\"eventType\":\"DROP_DATABASE\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"dropdb\",\"timestamp\":[0-9]+}"));
 
-    db = new Database("dropdb", "no description", "file:/tmp", emptyParameters);
+    // Parse the message field
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    assertEquals(EventType.DROP_DATABASE.toString(), jsonTree.get("eventType").asText());
+    assertEquals(dbName, jsonTree.get("db").asText());
+
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters);
     msClient.createDatabase(db);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.dropDatabase("dropdb");
+      msClient.dropDatabase(dbName2);
+      fail("Error: drop database should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
   }
 
   @Test
   public void createTable() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "createtable";
+    String tblName2 = "createtable2";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
     StorageDescriptor sd =
-        new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0, serde, null, null,
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
             emptyParameters);
-    Table table = new Table("mytable", "default", "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, null,
+            emptyParameters, null, null, null);
     msClient.createTable(table);
-    // Get the event
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_CREATE_TABLE_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("mytable", event.getTableName());
+    assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
 
     // Parse the message field
     ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
-    assertEquals(HCatConstants.HCAT_CREATE_TABLE_EVENT, jsonTree.get("eventType").asText());
-    assertEquals("default", jsonTree.get("db").asText());
-    assertEquals("mytable", jsonTree.get("table").asText());
+    assertEquals(EventType.CREATE_TABLE.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
     Table tableObj = JSONMessageFactory.getTableObj(jsonTree);
     assertEquals(table, tableObj);
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     table =
-        new Table("mytable2", "default", "me", startTime, startTime, 0, sd, null, emptyParameters,
-            null, null, null);
+        new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null,
+            emptyParameters, null, null, null);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
       msClient.createTable(table);
+      fail("Error: create table should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
   }
 
   @Test
   public void alterTable() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "altertabletbl";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
+    FieldSchema col2 = new FieldSchema("col2", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("alttable", "default", "me", startTime, startTime, 0, sd,
-        new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd,
+            new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
+
     // Event 1
     msClient.createTable(table);
-    cols.add(new FieldSchema("col2", "int", ""));
-    table = new Table("alttable", "default", "me", startTime, startTime, 0, sd,
-        new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
+    cols.add(col2);
+    table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd,
+            new ArrayList<FieldSchema>(), emptyParameters, null, null, null);
     // Event 2
-    msClient.alter_table("default", "alttable", table);
+    msClient.alter_table(defaultDbName, tblName, table);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_ALTER_TABLE_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("alttable", event.getTableName());
+    assertEquals(EventType.ALTER_TABLE.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
 
     // Parse the message field
     ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
-    assertEquals(HCatConstants.HCAT_ALTER_TABLE_EVENT, jsonTree.get("eventType").asText());
-    assertEquals("default", jsonTree.get("db").asText());
-    assertEquals("alttable", jsonTree.get("table").asText());
+    assertEquals(EventType.ALTER_TABLE.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
     Table tableObj = JSONMessageFactory.getTableObj(jsonTree);
     assertEquals(table, tableObj);
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.alter_table("default", "alttable", table);
+      msClient.alter_table(defaultDbName, tblName, table);
+      fail("Error: alter table should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
   }
 
   @Test
   public void dropTable() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "droptbl";
+    String tblName2 = "droptbl2";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("droptable", "default", "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, null,
+            emptyParameters, null, null, null);
+
+    // Event 1
     msClient.createTable(table);
-    msClient.dropTable("default", "droptable");
+    // Event 2
+    msClient.dropTable(defaultDbName, tblName);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_DROP_TABLE_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("droptable", event.getTableName());
-    assertTrue(event.getMessage().matches("\\{\"eventType\":\"DROP_TABLE\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":" +
-        "\"droptable\",\"timestamp\":[0-9]+}"));
-
-    table = new Table("droptable2", "default", "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    assertEquals(EventType.DROP_TABLE.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
+
+    // Parse the message field
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    assertEquals(EventType.DROP_TABLE.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
+
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    table =
+        new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null,
+            emptyParameters, null, null, null);
     msClient.createTable(table);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.dropTable("default", "droptable2");
+      msClient.dropTable(defaultDbName, tblName2);
+      fail("Error: drop table should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
   }
 
   @Test
   public void addPartition() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "addptn";
+    String tblName2 = "addptn2";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
-    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
-    partCols.add(new FieldSchema("ds", "string", ""));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("addPartTable", "default", "me", startTime, startTime, 0, sd, partCols,
-        emptyParameters, null, null, null);
-    msClient.createTable(table);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    FieldSchema partCol1 = new FieldSchema("ds", "string", "no comment");
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    List<String> partCol1Vals = Arrays.asList("today");
+    partCols.add(partCol1);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, partCols,
+            emptyParameters, null, null, null);
 
-    Partition partition = new Partition(Arrays.asList("today"), "default", "addPartTable",
-        startTime, startTime, sd, emptyParameters);
+    // Event 1
+    msClient.createTable(table);
+    Partition partition =
+        new Partition(partCol1Vals, defaultDbName, tblName, startTime, startTime, sd,
+            emptyParameters);
+    // Event 2
     msClient.add_partition(partition);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("addparttable", event.getTableName());
+    assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
 
     // Parse the message field
     ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, jsonTree.get("eventType").asText());
-    assertEquals("default", jsonTree.get("db").asText());
-    assertEquals("addparttable", jsonTree.get("table").asText());
+    assertEquals(EventType.ADD_PARTITION.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
     List<Partition> partitionObjList = JSONMessageFactory.getPartitionObjList(jsonTree);
     assertEquals(partition, partitionObjList.get(0));
 
-    partition = new Partition(Arrays.asList("tomorrow"), "default", "tableDoesNotExist",
-        startTime, startTime, sd, emptyParameters);
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    partition =
+        new Partition(Arrays.asList("tomorrow"), defaultDbName, tblName2, startTime, startTime, sd,
+            emptyParameters);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
       msClient.add_partition(partition);
+      fail("Error: add partition should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
   }
 
   @Test
   public void alterPartition() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "alterptn";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
-    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
-    partCols.add(new FieldSchema("ds", "string", ""));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("alterparttable", "default", "me", startTime, startTime, 0, sd,
-        partCols, emptyParameters, null, null, null);
-    msClient.createTable(table);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    FieldSchema partCol1 = new FieldSchema("ds", "string", "no comment");
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    List<String> partCol1Vals = Arrays.asList("today");
+    partCols.add(partCol1);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, partCols,
+            emptyParameters, null, null, null);
 
-    Partition partition = new Partition(Arrays.asList("today"), "default", "alterparttable",
-        startTime, startTime, sd, emptyParameters);
+    // Event 1
+    msClient.createTable(table);
+    Partition partition =
+        new Partition(partCol1Vals, defaultDbName, tblName, startTime, startTime, sd,
+            emptyParameters);
+    // Event 2
     msClient.add_partition(partition);
+    Partition newPart =
+        new Partition(Arrays.asList("today"), defaultDbName, tblName, startTime, startTime + 1, sd,
+            emptyParameters);
+    // Event 3
+    msClient.alter_partition(defaultDbName, tblName, newPart, null);
 
-    Partition newPart = new Partition(Arrays.asList("today"), "default", "alterparttable",
-        startTime, startTime + 1, sd, emptyParameters);
-    msClient.alter_partition("default", "alterparttable", newPart, null);
-
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("alterparttable", event.getTableName());
+    assertEquals(EventType.ALTER_PARTITION.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
 
     // Parse the message field
     ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
-    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, jsonTree.get("eventType").asText());
-    assertEquals("default", jsonTree.get("db").asText());
-    assertEquals("alterparttable", jsonTree.get("table").asText());
+    assertEquals(EventType.ALTER_PARTITION.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
     List<Partition> partitionObjList = JSONMessageFactory.getPartitionObjList(jsonTree);
     assertEquals(newPart, partitionObjList.get(0));
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.alter_partition("default", "alterparttable", newPart, null);
+      msClient.alter_partition(defaultDbName, tblName, newPart, null);
+      fail("Error: alter partition should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
   }
 
   @Test
   public void dropPartition() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "dropptn";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
-    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
-    partCols.add(new FieldSchema("ds", "string", ""));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("dropPartTable", "default", "me", startTime, startTime, 0, sd, partCols,
-        emptyParameters, null, null, null);
-    msClient.createTable(table);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    FieldSchema partCol1 = new FieldSchema("ds", "string", "no comment");
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    List<String> partCol1Vals = Arrays.asList("today");
+    partCols.add(partCol1);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, partCols,
+            emptyParameters, null, null, null);
 
-    Partition partition = new Partition(Arrays.asList("today"), "default", "dropPartTable",
-        startTime, startTime, sd, emptyParameters);
+    // Event 1
+    msClient.createTable(table);
+    Partition partition =
+        new Partition(partCol1Vals, defaultDbName, tblName, startTime, startTime, sd,
+            emptyParameters);
+    // Event 2
     msClient.add_partition(partition);
+    // Event 3
+    msClient.dropPartition(defaultDbName, tblName, partCol1Vals, false);
 
-    msClient.dropPartition("default", "dropparttable", Arrays.asList("today"), false);
-
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_DROP_PARTITION_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("dropparttable", event.getTableName());
-    assertTrue(event.getMessage().matches("\\{\"eventType\":\"DROP_PARTITION\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":" +
-        "\"dropparttable\",\"timestamp\":[0-9]+,\"partitions\":\\[\\{\"ds\":\"today\"}]}"));
-
-    partition = new Partition(Arrays.asList("tomorrow"), "default", "dropPartTable",
-        startTime, startTime, sd, emptyParameters);
-      msClient.add_partition(partition);
+    assertEquals(EventType.DROP_PARTITION.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
+
+    // Parse the message field
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    assertEquals(EventType.DROP_PARTITION.toString(), jsonTree.get("eventType").asText());
+    assertEquals(defaultDbName, jsonTree.get("db").asText());
+    assertEquals(tblName, jsonTree.get("table").asText());
+
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    List<String> newpartCol1Vals = Arrays.asList("tomorrow");
+    partition =
+        new Partition(newpartCol1Vals, defaultDbName, tblName, startTime, startTime, sd,
+            emptyParameters);
+    msClient.add_partition(partition);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.dropPartition("default", "dropparttable", Arrays.asList("tomorrow"), false);
+      msClient.dropPartition(defaultDbName, tblName, newpartCol1Vals, false);
+      fail("Error: drop partition should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(4, rsp.getEventsSize());
   }
 
   @Test
   public void createFunction() throws Exception {
-    String funcName = "createFunction";
-    String dbName = "default";
+    String defaultDbName = "default";
+    String funcName = "createfunction";
+    String funcName2 = "createfunction2";
     String ownerName = "me";
-    String funcClass = "o.a.h.h.myfunc";
+    String funcClass = "o.a.h.h.createfunc";
+    String funcClass2 = "o.a.h.h.createfunc2";
     String funcResource = "file:/tmp/somewhere";
-    Function func = new Function(funcName, dbName, funcClass, ownerName, PrincipalType.USER,
-        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
-        funcResource)));
+    String funcResource2 = "file:/tmp/somewhere2";
+    Function func =
+        new Function(funcName, defaultDbName, funcClass, ownerName, PrincipalType.USER, startTime,
+            FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, funcResource)));
+    // Event 1
     msClient.createFunction(func);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_CREATE_FUNCTION_EVENT, event.getEventType());
-    assertEquals(dbName, event.getDbName());
+    assertEquals(EventType.CREATE_FUNCTION.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+
+    // Parse the message field
     Function funcObj = JSONMessageFactory.getFunctionObj(JSONMessageFactory.getJsonTree(event));
-    assertEquals(dbName, funcObj.getDbName());
+    assertEquals(defaultDbName, funcObj.getDbName());
     assertEquals(funcName, funcObj.getFunctionName());
     assertEquals(funcClass, funcObj.getClassName());
     assertEquals(ownerName, funcObj.getOwnerName());
@@ -479,41 +602,53 @@ public class TestDbNotificationListener {
     assertEquals(ResourceType.JAR, funcObj.getResourceUris().get(0).getResourceType());
     assertEquals(funcResource, funcObj.getResourceUris().get(0).getUri());
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
-    func = new Function("createFunction2", dbName, "o.a.h.h.myfunc2", "me", PrincipalType.USER,
-        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
-        "file:/tmp/somewhere2")));
+    func =
+        new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER,
+            startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
+                funcResource2)));
     try {
       msClient.createFunction(func);
+      fail("Error: create function should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(1, rsp.getEventsSize());
   }
 
   @Test
   public void dropFunction() throws Exception {
-    String funcName = "dropfunctiontest";
-    String dbName = "default";
+    String defaultDbName = "default";
+    String funcName = "dropfunction";
+    String funcName2 = "dropfunction2";
     String ownerName = "me";
-    String funcClass = "o.a.h.h.dropFunctionTest";
+    String funcClass = "o.a.h.h.dropfunction";
+    String funcClass2 = "o.a.h.h.dropfunction2";
     String funcResource = "file:/tmp/somewhere";
-    Function func = new Function(funcName, dbName, funcClass, ownerName, PrincipalType.USER,
-        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
-        funcResource)));
+    String funcResource2 = "file:/tmp/somewhere2";
+    Function func =
+        new Function(funcName, defaultDbName, funcClass, ownerName, PrincipalType.USER, startTime,
+            FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, funcResource)));
+    // Event 1
     msClient.createFunction(func);
-    msClient.dropFunction(dbName, funcName);
+    // Event 2
+    msClient.dropFunction(defaultDbName, funcName);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_DROP_FUNCTION_EVENT, event.getEventType());
-    assertEquals(dbName, event.getDbName());
+    assertEquals(EventType.DROP_FUNCTION.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+
+    // Parse the message field
     Function funcObj = JSONMessageFactory.getFunctionObj(JSONMessageFactory.getJsonTree(event));
-    assertEquals(dbName, funcObj.getDbName());
+    assertEquals(defaultDbName, funcObj.getDbName());
     assertEquals(funcName, funcObj.getFunctionName());
     assertEquals(funcClass, funcObj.getClassName());
     assertEquals(ownerName, funcObj.getOwnerName());
@@ -522,17 +657,20 @@ public class TestDbNotificationListener {
     assertEquals(ResourceType.JAR, funcObj.getResourceUris().get(0).getResourceType());
     assertEquals(funcResource, funcObj.getResourceUris().get(0).getUri());
 
-    func = new Function("dropfunctiontest2", dbName, "o.a.h.h.dropFunctionTest2", "me",
-        PrincipalType.USER,  startTime, FunctionType.JAVA, Arrays.asList(
-        new ResourceUri(ResourceType.JAR, "file:/tmp/somewhere2")));
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    func =
+        new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER,
+            startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
+                funcResource2)));
     msClient.createFunction(func);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.dropFunction(dbName, "dropfunctiontest2");
+      msClient.dropFunction(defaultDbName, funcName2);
+      fail("Error: drop function should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
   }
@@ -543,46 +681,60 @@ public class TestDbNotificationListener {
     String dbName = "default";
     String tableName = "createIndexTable";
     String indexTableName = tableName + "__" + indexName + "__";
-    int startTime = (int)(System.currentTimeMillis() / 1000);
+    int startTime = (int) (System.currentTimeMillis() / 1000);
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", "int", ""));
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
-    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17, serde,
+            Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table =
+        new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 1
     msClient.createTable(table);
-    Index index = new Index(indexName, null, "default", tableName, startTime, startTime,
-        indexTableName, sd, emptyParameters, false);
-    Table indexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    Index index =
+        new Index(indexName, null, "default", tableName, startTime, startTime, indexTableName, sd,
+            emptyParameters, false);
+    Table indexTable =
+        new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 2, 3 (index table and index)
     msClient.createIndex(index, indexTable);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_CREATE_INDEX_EVENT, event.getEventType());
+    assertEquals(EventType.CREATE_INDEX.toString(), event.getEventType());
     assertEquals(dbName, event.getDbName());
+
+    // Parse the message field
     Index indexObj = JSONMessageFactory.getIndexObj(JSONMessageFactory.getJsonTree(event));
     assertEquals(dbName, indexObj.getDbName());
     assertEquals(indexName, indexObj.getIndexName());
     assertEquals(tableName, indexObj.getOrigTableName());
     assertEquals(indexTableName, indexObj.getIndexTableName());
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
-    index = new Index("createIndexTable2", null, "default", tableName, startTime, startTime,
-        "createIndexTable2__createIndexTable2__", sd, emptyParameters, false);
-    Table indexTable2 = new Table("createIndexTable2__createIndexTable2__", dbName, "me",
-        startTime, startTime, 0, sd, null, emptyParameters, null, null, null);
+    index =
+        new Index("createIndexTable2", null, "default", tableName, startTime, startTime,
+            "createIndexTable2__createIndexTable2__", sd, emptyParameters, false);
+    Table indexTable2 =
+        new Table("createIndexTable2__createIndexTable2__", dbName, "me", startTime, startTime, 0,
+            sd, null, emptyParameters, null, null, null);
     try {
       msClient.createIndex(index, indexTable2);
+      fail("Error: create index should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
   }
@@ -593,44 +745,61 @@ public class TestDbNotificationListener {
     String dbName = "default";
     String tableName = "dropIndexTable";
     String indexTableName = tableName + "__" + indexName + "__";
-    int startTime = (int)(System.currentTimeMillis() / 1000);
+    int startTime = (int) (System.currentTimeMillis() / 1000);
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", "int", ""));
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
-    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17, serde,
+            Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table =
+        new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 1
     msClient.createTable(table);
-    Index index = new Index(indexName, null, "default", tableName, startTime, startTime,
-        indexTableName, sd, emptyParameters, false);
-    Table indexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    Index index =
+        new Index(indexName, null, "default", tableName, startTime, startTime, indexTableName, sd,
+            emptyParameters, false);
+    Table indexTable =
+        new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 2, 3 (index table and index)
     msClient.createIndex(index, indexTable);
-    msClient.dropIndex(dbName, tableName, indexName, true); // drops index and indexTable
+    // Event 4 (drops index and indexTable)
+    msClient.dropIndex(dbName, tableName, indexName, true);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(4, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(3);
     assertEquals(firstEventId + 4, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_DROP_INDEX_EVENT, event.getEventType());
+    assertEquals(EventType.DROP_INDEX.toString(), event.getEventType());
     assertEquals(dbName, event.getDbName());
+
+    // Parse the message field
     Index indexObj = JSONMessageFactory.getIndexObj(JSONMessageFactory.getJsonTree(event));
     assertEquals(dbName, indexObj.getDbName());
     assertEquals(indexName.toLowerCase(), indexObj.getIndexName());
     assertEquals(tableName.toLowerCase(), indexObj.getOrigTableName());
     assertEquals(indexTableName.toLowerCase(), indexObj.getIndexTableName());
 
-    index = new Index("dropIndexTable2", null, "default", tableName, startTime, startTime,
-        "dropIndexTable__dropIndexTable2__", sd, emptyParameters, false);
-    Table indexTable2 = new Table("dropIndexTable__dropIndexTable2__", dbName, "me", startTime,
-        startTime, 0, sd, null, emptyParameters, null, null, null);
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
+    index =
+        new Index("dropIndexTable2", null, "default", tableName, startTime, startTime,
+            "dropIndexTable__dropIndexTable2__", sd, emptyParameters, false);
+    Table indexTable2 =
+        new Table("dropIndexTable__dropIndexTable2__", dbName, "me", startTime, startTime, 0, sd,
+            null, emptyParameters, null, null, null);
     msClient.createIndex(index, indexTable2);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
-      msClient.dropIndex(dbName, tableName, "dropIndex2", true); // drops index and indexTable
+      // drops index and indexTable
+      msClient.dropIndex(dbName, tableName, "dropIndex2", true);
+      fail("Error: drop index should've failed");
     } catch (Exception ex) {
       // expected
     }
@@ -645,127 +814,165 @@ public class TestDbNotificationListener {
     String dbName = "default";
     String tableName = "alterIndexTable";
     String indexTableName = tableName + "__" + indexName + "__";
-    int startTime = (int)(System.currentTimeMillis() / 1000);
+    int startTime = (int) (System.currentTimeMillis() / 1000);
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", "int", ""));
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
-    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17, serde,
+            Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table =
+        new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 1
     msClient.createTable(table);
-    Index oldIndex = new Index(indexName, null, "default", tableName, startTime, startTime,
-        indexTableName, sd, emptyParameters, false);
-    Table oldIndexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    Index oldIndex =
+        new Index(indexName, null, "default", tableName, startTime, startTime, indexTableName, sd,
+            emptyParameters, false);
+    Table oldIndexTable =
+        new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null, emptyParameters,
+            null, null, null);
+    // Event 2, 3
     msClient.createIndex(oldIndex, oldIndexTable); // creates index and index table
-    Index newIndex = new Index(indexName, null, "default", tableName, startTime, startTime + 1,
-        indexTableName, sd, emptyParameters, false);
+    Index newIndex =
+        new Index(indexName, null, "default", tableName, startTime, startTime + 1, indexTableName,
+            sd, emptyParameters, false);
+    // Event 4
     msClient.alter_index(dbName, tableName, indexName, newIndex);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(4, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(3);
     assertEquals(firstEventId + 4, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_ALTER_INDEX_EVENT, event.getEventType());
+    assertEquals(EventType.ALTER_INDEX.toString(), event.getEventType());
     assertEquals(dbName, event.getDbName());
-    Index indexObj = JSONMessageFactory.getIndexObj(JSONMessageFactory.getJsonTree(event), "afterIndexObjJson");
+
+    // Parse the message field
+    Index indexObj =
+        JSONMessageFactory.getIndexObj(JSONMessageFactory.getJsonTree(event), "afterIndexObjJson");
     assertEquals(dbName, indexObj.getDbName());
     assertEquals(indexName, indexObj.getIndexName());
     assertEquals(tableName, indexObj.getOrigTableName());
     assertEquals(indexTableName, indexObj.getIndexTableName());
     assertTrue(indexObj.getCreateTime() < indexObj.getLastAccessTime());
 
+    // When hive.metastore.transactional.event.listeners is set,
+    // a failed event should not create a new notification
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
       msClient.alter_index(dbName, tableName, indexName, newIndex);
+      fail("Error: alter index should've failed");
     } catch (Exception ex) {
       // expected
     }
-
     rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(4, rsp.getEventsSize());
   }
 
   @Test
   public void insertTable() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "inserttbl";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    String fileAdded = "/warehouse/mytable/b1";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("insertTable", "default", "me", startTime, startTime, 0, sd, null,
-        emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, null,
+            emptyParameters, null, null, null);
+    // Event 1
     msClient.createTable(table);
 
     FireEventRequestData data = new FireEventRequestData();
     InsertEventRequestData insertData = new InsertEventRequestData();
     data.setInsertData(insertData);
-    insertData.addToFilesAdded("/warehouse/mytable/b1");
+    insertData.addToFilesAdded(fileAdded);
     FireEventRequest rqst = new FireEventRequest(true, data);
-    rqst.setDbName("default");
-    rqst.setTableName("insertTable");
+    rqst.setDbName(defaultDbName);
+    rqst.setTableName(tblName);
+    // Event 2
     msClient.fireListenerEvent(rqst);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("insertTable", event.getTableName());
-    assertTrue(event.getMessage(),
-        event.getMessage().matches("\\{\"eventType\":\"INSERT\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":" +
-        "\"insertTable\",\"timestamp\":[0-9]+,\"files\":\\[\"/warehouse/mytable/b1\"]," +
-        "\"partKeyVals\":\\{},\"partitionKeyValues\":\\{}}"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
+    // Parse the message field
+    verifyInsertJSON(event, defaultDbName, tblName, false);
   }
 
   @Test
   public void insertPartition() throws Exception {
+    String defaultDbName = "default";
+    String tblName = "insertptn";
+    String tblOwner = "me";
+    String serdeLocation = "file:/tmp";
+    String fileAdded = "/warehouse/mytable/b1";
+    FieldSchema col1 = new FieldSchema("col1", "int", "no comment");
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("col1", "int", "nocomment"));
-    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
-    partCols.add(new FieldSchema("ds", "string", ""));
+    cols.add(col1);
     SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
-    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
-        serde, null, null, emptyParameters);
-    Table table = new Table("insertPartition", "default", "me", startTime, startTime, 0, sd,
-        partCols, emptyParameters, null, null, null);
+    StorageDescriptor sd =
+        new StorageDescriptor(cols, serdeLocation, "input", "output", false, 0, serde, null, null,
+            emptyParameters);
+    FieldSchema partCol1 = new FieldSchema("ds", "string", "no comment");
+    List<FieldSchema> partCols = new ArrayList<FieldSchema>();
+    List<String> partCol1Vals = Arrays.asList("today");
+    LinkedHashMap<String, String> partKeyVals = new LinkedHashMap<String, String>();
+    partKeyVals.put("ds", "today");
+    partCols.add(partCol1);
+    Table table =
+        new Table(tblName, defaultDbName, tblOwner, startTime, startTime, 0, sd, partCols,
+            emptyParameters, null, null, null);
+    // Event 1
     msClient.createTable(table);
-    Partition partition = new Partition(Arrays.asList("today"), "default", "insertPartition",
-        startTime, startTime, sd, emptyParameters);
+    Partition partition =
+        new Partition(partCol1Vals, defaultDbName, tblName, startTime, startTime, sd,
+            emptyParameters);
+    // Event 2
     msClient.add_partition(partition);
-
     FireEventRequestData data = new FireEventRequestData();
     InsertEventRequestData insertData = new InsertEventRequestData();
     data.setInsertData(insertData);
-    insertData.addToFilesAdded("/warehouse/mytable/today/b1");
+    insertData.addToFilesAdded(fileAdded);
     FireEventRequest rqst = new FireEventRequest(true, data);
-    rqst.setDbName("default");
-    rqst.setTableName("insertPartition");
-    rqst.setPartitionVals(Arrays.asList("today"));
+    rqst.setDbName(defaultDbName);
+    rqst.setTableName(tblName);
+    rqst.setPartitionVals(partCol1Vals);
+    // Event 3
     msClient.fireListenerEvent(rqst);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(3, rsp.getEventsSize());
-
     NotificationEvent event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
     assertTrue(event.getEventTime() >= startTime);
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertEquals("default", event.getDbName());
-    assertEquals("insertPartition", event.getTableName());
-    assertTrue(event.getMessage(),
-        event.getMessage().matches("\\{\"eventType\":\"INSERT\",\"server\":\"\"," +
-        "\"servicePrincipal\":\"\",\"db\":\"default\",\"table\":" +
-        "\"insertPartition\",\"timestamp\":[0-9]+," +
-        "\"files\":\\[\"/warehouse/mytable/today/b1\"],\"partKeyVals\":\\{\"ds\":\"today\"}," +
-        "\"partitionKeyValues\":\\{\"ds\":\"today\"}}"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    assertEquals(defaultDbName, event.getDbName());
+    assertEquals(tblName, event.getTableName());
+    // Parse the message field
+    verifyInsertJSON(event, defaultDbName, tblName, false);
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    LinkedHashMap<String, String> partKeyValsFromNotif =
+        JSONMessageFactory.getAsMap((ObjectNode) jsonTree.get("partKeyVals"),
+            new LinkedHashMap<String, String>());
+    assertEquals(partKeyVals, partKeyValsFromNotif);
   }
 
   @Test
@@ -777,6 +984,7 @@ public class TestDbNotificationListener {
     db = new Database("db3", "no description", "file:/tmp", emptyParameters);
     msClient.createDatabase(db);
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 2, null);
     assertEquals(2, rsp.getEventsSize());
     assertEquals(firstEventId + 1, rsp.getEvents().get(0).getEventId());
@@ -794,10 +1002,11 @@ public class TestDbNotificationListener {
     IMetaStoreClient.NotificationFilter filter = new IMetaStoreClient.NotificationFilter() {
       @Override
       public boolean accept(NotificationEvent event) {
-        return event.getEventType().equals(HCatConstants.HCAT_DROP_DATABASE_EVENT);
+        return event.getEventType().equals(EventType.DROP_DATABASE.toString());
       }
     };
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, filter);
     assertEquals(1, rsp.getEventsSize());
     assertEquals(firstEventId + 3, rsp.getEvents().get(0).getEventId());
@@ -814,10 +1023,11 @@ public class TestDbNotificationListener {
     IMetaStoreClient.NotificationFilter filter = new IMetaStoreClient.NotificationFilter() {
       @Override
       public boolean accept(NotificationEvent event) {
-        return event.getEventType().equals(HCatConstants.HCAT_CREATE_DATABASE_EVENT);
+        return event.getEventType().equals(EventType.CREATE_DATABASE.toString());
       }
     };
 
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 1, filter);
     assertEquals(1, rsp.getEventsSize());
     assertEquals(firstEventId + 1, rsp.getEvents().get(0).getEventId());
@@ -825,161 +1035,223 @@ public class TestDbNotificationListener {
 
   @Test
   public void sqlInsertTable() throws Exception {
-
-    driver.run("create table sit (c int)");
-    driver.run("insert into table sit values (1)");
-    driver.run("alter table sit add columns (c2 int)");
-    driver.run("drop table sit");
-
+    String defaultDbName = "default";
+    String tblName = "sqlins";
+    // Event 1
+    driver.run("create table " + tblName + " (c int)");
+    // Event 2 (alter: marker stats event), 3 (insert), 4 (alter: stats update event)
+    driver.run("insert into table " + tblName + " values (1)");
+    // Event 5
+    driver.run("alter table " + tblName + " add columns (c2 int)");
+    // Event 6
+    driver.run("drop table " + tblName);
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
-    // For reasons not clear to me there's an alter after the create table and one after the
-    // insert.  I think the one after the insert is a stats calculation.
     assertEquals(6, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
-    assertEquals(HCatConstants.HCAT_CREATE_TABLE_EVENT, event.getEventType());
+    assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
+
     event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    // Make sure the files are listed in the insert
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, defaultDbName, tblName, true);
+
     event = rsp.getEvents().get(4);
     assertEquals(firstEventId + 5, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ALTER_TABLE_EVENT, event.getEventType());
+    assertEquals(EventType.ALTER_TABLE.toString(), event.getEventType());
+
     event = rsp.getEvents().get(5);
     assertEquals(firstEventId + 6, event.getEventId());
-    assertEquals(HCatConstants.HCAT_DROP_TABLE_EVENT, event.getEventType());
+    assertEquals(EventType.DROP_TABLE.toString(), event.getEventType());
   }
 
   @Test
   public void sqlCTAS() throws Exception {
+    String sourceTblName = "sqlctasins1";
+    String targetTblName = "sqlctasins2";
+    // Event 1
+    driver.run("create table " + sourceTblName + " (c int)");
+    // Event 2 (alter: marker stats event), 3 (insert), 4 (alter: stats update event)
+    driver.run("insert into table " + sourceTblName + " values (1)");
+    // Event 5, 6 (alter: stats update event)
+    driver.run("create table " + targetTblName + " as select c from " + sourceTblName);
 
-    driver.run("create table ctas_source (c int)");
-    driver.run("insert into table ctas_source values (1)");
-    driver.run("create table ctas_target as select c from ctas_source");
-
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
-
     assertEquals(6, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
-    assertEquals(HCatConstants.HCAT_CREATE_TABLE_EVENT, event.getEventType());
+    assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
+
     event = rsp.getEvents().get(2);
     assertEquals(firstEventId + 3, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    // Make sure the files are listed in the insert
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, null, sourceTblName, true);
+
     event = rsp.getEvents().get(4);
     assertEquals(firstEventId + 5, event.getEventId());
-    assertEquals(HCatConstants.HCAT_CREATE_TABLE_EVENT, event.getEventType());
+    assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
   }
 
   @Test
   public void sqlTempTable() throws Exception {
+    String tempTblName = "sqltemptbl";
+    driver.run("create temporary table " + tempTblName + "  (c int)");
+    driver.run("insert into table " + tempTblName + " values (1)");
 
-    LOG.info("XXX Starting temp table");
-    driver.run("create temporary table tmp1 (c int)");
-    driver.run("insert into table tmp1 values (1)");
-
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
-
     assertEquals(0, rsp.getEventsSize());
   }
 
   @Test
   public void sqlDb() throws Exception {
+    String dbName = "sqldb";
+    // Event 1
+    driver.run("create database " + dbName);
+    // Event 2
+    driver.run("drop database " + dbName);
 
-    driver.run("create database sd");
-    driver.run("drop database sd");
-
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
     assertEquals(2, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
-    assertEquals(HCatConstants.HCAT_CREATE_DATABASE_EVENT, event.getEventType());
+    assertEquals(EventType.CREATE_DATABASE.toString(), event.getEventType());
     event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
-    assertEquals(HCatConstants.HCAT_DROP_DATABASE_EVENT, event.getEventType());
+    assertEquals(EventType.DROP_DATABASE.toString(), event.getEventType());
   }
 
   @Test
   public void sqlInsertPartition() throws Exception {
-
-    driver.run("create table sip (c int) partitioned by (ds string)");
-    driver.run("insert into table sip partition (ds = 'today') values (1)");
-    driver.run("insert into table sip partition (ds = 'today') values (2)");
-    driver.run("insert into table sip partition (ds) values (3, 'today')");
-    driver.run("alter table sip add partition (ds = 'yesterday')");
-    driver.run("insert into table sip partition (ds = 'yesterday') values (2)");
-
-    driver.run("insert into table sip partition (ds) values (3, 'yesterday')");
-    driver.run("insert into table sip partition (ds) values (3, 'tomorrow')");
-    driver.run("alter table sip drop partition (ds = 'tomorrow')");
-
-    driver.run("insert into table sip partition (ds) values (42, 'todaytwo')");
-    driver.run("insert overwrite table sip partition(ds='todaytwo') select c from sip where 'ds'='today'");
-
+    String tblName = "sqlinsptn";
+    // Event 1
+    driver.run("create table " + tblName + " (c int) partitioned by (ds string)");
+    // Event 2, 3, 4
+    driver.run("insert into table " + tblName + " partition (ds = 'today') values (1)");
+    // Event 5, 6, 7
+    driver.run("insert into table " + tblName + " partition (ds = 'today') values (2)");
+    // Event 8, 9, 10
+    driver.run("insert into table " + tblName + " partition (ds) values (3, 'today')");
+    // Event 9, 10
+    driver.run("alter table " + tblName + " add partition (ds = 'yesterday')");
+    // Event 10, 11, 12
+    driver.run("insert into table " + tblName + " partition (ds = 'yesterday') values (2)");
+    // Event 12, 13, 14
+    driver.run("insert into table " + tblName + " partition (ds) values (3, 'yesterday')");
+    // Event 15, 16, 17
+    driver.run("insert into table " + tblName + " partition (ds) values (3, 'tomorrow')");
+    // Event 18
+    driver.run("alter table " + tblName + " drop partition (ds = 'tomorrow')");
+    // Event 19, 20, 21
+    driver.run("insert into table " + tblName + " partition (ds) values (42, 'todaytwo')");
+    // Event 22, 23, 24
+    driver.run("insert overwrite table " + tblName + " partition(ds='todaytwo') select c from "
+        + tblName + " where 'ds'='today'");
+
+    // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
-
-    for (NotificationEvent ne : rsp.getEvents()) LOG.debug("EVENT: " + ne.getMessage());
-    // For reasons not clear to me there's one or more alter partitions after add partition and
-    // insert.
     assertEquals(24, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(1);
     assertEquals(firstEventId + 2, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType());
+
     event = rsp.getEvents().get(3);
     assertEquals(firstEventId + 4, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    // Make sure the files are listed in the insert
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, null, tblName, true);
+
     event = rsp.getEvents().get(6);
     assertEquals(firstEventId + 7, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, null, tblName, true);
+
     event = rsp.getEvents().get(9);
     assertEquals(firstEventId + 10, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType());
+
     event = rsp.getEvents().get(10);
     assertEquals(firstEventId + 11, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, null, tblName, true);
+
     event = rsp.getEvents().get(13);
     assertEquals(firstEventId + 14, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\"pfile.*"));
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // Parse the message field
+    verifyInsertJSON(event, null, tblName, true);
+
     event = rsp.getEvents().get(16);
     assertEquals(firstEventId + 17, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType());
+
     event = rsp.getEvents().get(18);
     assertEquals(firstEventId + 19, event.getEventId());
-    assertEquals(HCatConstants.HCAT_DROP_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.DROP_PARTITION.toString(), event.getEventType());
 
     event = rsp.getEvents().get(19);
     assertEquals(firstEventId + 20, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ADD_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType());
+
     event = rsp.getEvents().get(20);
     assertEquals(firstEventId + 21, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ALTER_PARTITION.toString(), event.getEventType());
     assertTrue(event.getMessage().matches(".*\"ds\":\"todaytwo\".*"));
 
     event = rsp.getEvents().get(21);
     assertEquals(firstEventId + 22, event.getEventId());
-    assertEquals(HCatConstants.HCAT_INSERT_EVENT, event.getEventType());
-    assertTrue(event.getMessage().matches(".*\"files\":\\[\\].*")); // replace-overwrite introduces no new files
+    assertEquals(EventType.INSERT.toString(), event.getEventType());
+    // replace-overwrite introduces no new files
+    assertTrue(event.getMessage().matches(".*\"files\":\\[\\].*"));
+
     event = rsp.getEvents().get(22);
     assertEquals(firstEventId + 23, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ALTER_PARTITION.toString(), event.getEventType());
     assertTrue(event.getMessage().matches(".*\"ds\":\"todaytwo\".*"));
+
     event = rsp.getEvents().get(23);
     assertEquals(firstEventId + 24, event.getEventId());
-    assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
+    assertEquals(EventType.ALTER_PARTITION.toString(), event.getEventType());
     assertTrue(event.getMessage().matches(".*\"ds\":\"todaytwo\".*"));
-   }
+  }
+
+  private void verifyInsertJSON(NotificationEvent event, String dbName, String tblName,
+      boolean verifyChecksums) throws Exception {
+    // Parse the message field
+    ObjectNode jsonTree = JSONMessageFactory.getJsonTree(event);
+    System.out.println("JSONInsertMessage: " + jsonTree.toString());
+    assertEquals(EventType.INSERT.toString(), jsonTree.get("eventType").asText());
+    if (dbName != null) {
+      assertEquals(dbName, jsonTree.get("db").asText());
+    }
+    if (tblName != null) {
+      assertEquals(tblName, jsonTree.get("table").asText());
+    }
+    // Should have list of files
+    List<String> files =
+        JSONMessageFactory.getAsList((ArrayNode) jsonTree.get("files"), new ArrayList<String>());
+    assertTrue(files.size() > 0);
+    if (verifyChecksums) {
+      // Should have list of file checksums
+      List<String> fileChecksums =
+          JSONMessageFactory.getAsList((ArrayNode) jsonTree.get("fileChecksums"),
+              new ArrayList<String>());
+      assertTrue(fileChecksums.size() > 0);
+
+    }
+  }
 
   @Test
   public void cleanupNotifs() throws Exception {
-    Database db = new Database("cleanup1","no description","file:/tmp", emptyParameters);
+    Database db = new Database("cleanup1", "no description", "file:/tmp", emptyParameters);
     msClient.createDatabase(db);
     msClient.dropDatabase("cleanup1");
 
@@ -988,7 +1260,8 @@ public class TestDbNotificationListener {
     assertEquals(2, rsp.getEventsSize());
 
     // sleep for expiry time, and then fetch again
-    Thread.sleep(EVENTS_TTL * 2 * 1000); // sleep twice the TTL interval - things should have been cleaned by then.
+    // sleep twice the TTL interval - things should have been cleaned by then.
+    Thread.sleep(EVENTS_TTL * 2 * 1000);
 
     LOG.info("Pulling events again after cleanup");
     NotificationEventResponse rsp2 = msClient.getNextNotification(firstEventId, 0, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 6f77156..79592ea 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -811,7 +811,9 @@ struct CurrentNotificationEventId {
 }
 
 struct InsertEventRequestData {
-    1: required list<string> filesAdded
+    1: required list<string> filesAdded,
+    // Checksum of files (UTF8 encoded string) added during this insert event (at the time they were added)
+    2: optional list<binary> filesAddedChecksum,
 }
 
 union FireEventRequestData {


[2/8] hive git commit: HIVE-15294: Capture additional metadata to replicate a simple insert at destination (Vaibhav Gumashta reviewed by Thejas Nair)

Posted by vg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 595c448..103cd86 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -15987,6 +15987,10 @@ class InsertEventRequestData {
    * @var string[]
    */
   public $filesAdded = null;
+  /**
+   * @var string[]
+   */
+  public $filesAddedChecksum = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -15999,12 +16003,23 @@ class InsertEventRequestData {
             'type' => TType::STRING,
             ),
           ),
+        2 => array(
+          'var' => 'filesAddedChecksum',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
       if (isset($vals['filesAdded'])) {
         $this->filesAdded = $vals['filesAdded'];
       }
+      if (isset($vals['filesAddedChecksum'])) {
+        $this->filesAddedChecksum = $vals['filesAddedChecksum'];
+      }
     }
   }
 
@@ -16044,6 +16059,23 @@ class InsertEventRequestData {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 2:
+          if ($ftype == TType::LST) {
+            $this->filesAddedChecksum = array();
+            $_size501 = 0;
+            $_etype504 = 0;
+            $xfer += $input->readListBegin($_etype504, $_size501);
+            for ($_i505 = 0; $_i505 < $_size501; ++$_i505)
+            {
+              $elem506 = null;
+              $xfer += $input->readString($elem506);
+              $this->filesAddedChecksum []= $elem506;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -16065,9 +16097,26 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter501)
+          foreach ($this->filesAdded as $iter507)
+          {
+            $xfer += $output->writeString($iter507);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->filesAddedChecksum !== null) {
+      if (!is_array($this->filesAddedChecksum)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('filesAddedChecksum', TType::LST, 2);
+      {
+        $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
+        {
+          foreach ($this->filesAddedChecksum as $iter508)
           {
-            $xfer += $output->writeString($iter501);
+            $xfer += $output->writeString($iter508);
           }
         }
         $output->writeListEnd();
@@ -16285,14 +16334,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size502 = 0;
-            $_etype505 = 0;
-            $xfer += $input->readListBegin($_etype505, $_size502);
-            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
+            $_size509 = 0;
+            $_etype512 = 0;
+            $xfer += $input->readListBegin($_etype512, $_size509);
+            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
             {
-              $elem507 = null;
-              $xfer += $input->readString($elem507);
-              $this->partitionVals []= $elem507;
+              $elem514 = null;
+              $xfer += $input->readString($elem514);
+              $this->partitionVals []= $elem514;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16343,9 +16392,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter508)
+          foreach ($this->partitionVals as $iter515)
           {
-            $xfer += $output->writeString($iter508);
+            $xfer += $output->writeString($iter515);
           }
         }
         $output->writeListEnd();
@@ -16573,18 +16622,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size509 = 0;
-            $_ktype510 = 0;
-            $_vtype511 = 0;
-            $xfer += $input->readMapBegin($_ktype510, $_vtype511, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size516 = 0;
+            $_ktype517 = 0;
+            $_vtype518 = 0;
+            $xfer += $input->readMapBegin($_ktype517, $_vtype518, $_size516);
+            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
             {
-              $key514 = 0;
-              $val515 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key514);
-              $val515 = new \metastore\MetadataPpdResult();
-              $xfer += $val515->read($input);
-              $this->metadata[$key514] = $val515;
+              $key521 = 0;
+              $val522 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key521);
+              $val522 = new \metastore\MetadataPpdResult();
+              $xfer += $val522->read($input);
+              $this->metadata[$key521] = $val522;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16619,10 +16668,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter516 => $viter517)
+          foreach ($this->metadata as $kiter523 => $viter524)
           {
-            $xfer += $output->writeI64($kiter516);
-            $xfer += $viter517->write($output);
+            $xfer += $output->writeI64($kiter523);
+            $xfer += $viter524->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16724,14 +16773,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size518 = 0;
-            $_etype521 = 0;
-            $xfer += $input->readListBegin($_etype521, $_size518);
-            for ($_i522 = 0; $_i522 < $_size518; ++$_i522)
+            $_size525 = 0;
+            $_etype528 = 0;
+            $xfer += $input->readListBegin($_etype528, $_size525);
+            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
             {
-              $elem523 = null;
-              $xfer += $input->readI64($elem523);
-              $this->fileIds []= $elem523;
+              $elem530 = null;
+              $xfer += $input->readI64($elem530);
+              $this->fileIds []= $elem530;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16780,9 +16829,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter524)
+          foreach ($this->fileIds as $iter531)
           {
-            $xfer += $output->writeI64($iter524);
+            $xfer += $output->writeI64($iter531);
           }
         }
         $output->writeListEnd();
@@ -16876,17 +16925,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size525 = 0;
-            $_ktype526 = 0;
-            $_vtype527 = 0;
-            $xfer += $input->readMapBegin($_ktype526, $_vtype527, $_size525);
-            for ($_i529 = 0; $_i529 < $_size525; ++$_i529)
+            $_size532 = 0;
+            $_ktype533 = 0;
+            $_vtype534 = 0;
+            $xfer += $input->readMapBegin($_ktype533, $_vtype534, $_size532);
+            for ($_i536 = 0; $_i536 < $_size532; ++$_i536)
             {
-              $key530 = 0;
-              $val531 = '';
-              $xfer += $input->readI64($key530);
-              $xfer += $input->readString($val531);
-              $this->metadata[$key530] = $val531;
+              $key537 = 0;
+              $val538 = '';
+              $xfer += $input->readI64($key537);
+              $xfer += $input->readString($val538);
+              $this->metadata[$key537] = $val538;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16921,10 +16970,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter532 => $viter533)
+          foreach ($this->metadata as $kiter539 => $viter540)
           {
-            $xfer += $output->writeI64($kiter532);
-            $xfer += $output->writeString($viter533);
+            $xfer += $output->writeI64($kiter539);
+            $xfer += $output->writeString($viter540);
           }
         }
         $output->writeMapEnd();
@@ -16993,14 +17042,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size534 = 0;
-            $_etype537 = 0;
-            $xfer += $input->readListBegin($_etype537, $_size534);
-            for ($_i538 = 0; $_i538 < $_size534; ++$_i538)
+            $_size541 = 0;
+            $_etype544 = 0;
+            $xfer += $input->readListBegin($_etype544, $_size541);
+            for ($_i545 = 0; $_i545 < $_size541; ++$_i545)
             {
-              $elem539 = null;
-              $xfer += $input->readI64($elem539);
-              $this->fileIds []= $elem539;
+              $elem546 = null;
+              $xfer += $input->readI64($elem546);
+              $this->fileIds []= $elem546;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17028,9 +17077,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter540)
+          foreach ($this->fileIds as $iter547)
           {
-            $xfer += $output->writeI64($iter540);
+            $xfer += $output->writeI64($iter547);
           }
         }
         $output->writeListEnd();
@@ -17170,14 +17219,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size541 = 0;
-            $_etype544 = 0;
-            $xfer += $input->readListBegin($_etype544, $_size541);
-            for ($_i545 = 0; $_i545 < $_size541; ++$_i545)
+            $_size548 = 0;
+            $_etype551 = 0;
+            $xfer += $input->readListBegin($_etype551, $_size548);
+            for ($_i552 = 0; $_i552 < $_size548; ++$_i552)
             {
-              $elem546 = null;
-              $xfer += $input->readI64($elem546);
-              $this->fileIds []= $elem546;
+              $elem553 = null;
+              $xfer += $input->readI64($elem553);
+              $this->fileIds []= $elem553;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17187,14 +17236,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size547 = 0;
-            $_etype550 = 0;
-            $xfer += $input->readListBegin($_etype550, $_size547);
-            for ($_i551 = 0; $_i551 < $_size547; ++$_i551)
+            $_size554 = 0;
+            $_etype557 = 0;
+            $xfer += $input->readListBegin($_etype557, $_size554);
+            for ($_i558 = 0; $_i558 < $_size554; ++$_i558)
             {
-              $elem552 = null;
-              $xfer += $input->readString($elem552);
-              $this->metadata []= $elem552;
+              $elem559 = null;
+              $xfer += $input->readString($elem559);
+              $this->metadata []= $elem559;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17229,9 +17278,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter553)
+          foreach ($this->fileIds as $iter560)
           {
-            $xfer += $output->writeI64($iter553);
+            $xfer += $output->writeI64($iter560);
           }
         }
         $output->writeListEnd();
@@ -17246,9 +17295,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter554)
+          foreach ($this->metadata as $iter561)
           {
-            $xfer += $output->writeString($iter554);
+            $xfer += $output->writeString($iter561);
           }
         }
         $output->writeListEnd();
@@ -17367,14 +17416,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size555 = 0;
-            $_etype558 = 0;
-            $xfer += $input->readListBegin($_etype558, $_size555);
-            for ($_i559 = 0; $_i559 < $_size555; ++$_i559)
+            $_size562 = 0;
+            $_etype565 = 0;
+            $xfer += $input->readListBegin($_etype565, $_size562);
+            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
             {
-              $elem560 = null;
-              $xfer += $input->readI64($elem560);
-              $this->fileIds []= $elem560;
+              $elem567 = null;
+              $xfer += $input->readI64($elem567);
+              $this->fileIds []= $elem567;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17402,9 +17451,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter561)
+          foreach ($this->fileIds as $iter568)
           {
-            $xfer += $output->writeI64($iter561);
+            $xfer += $output->writeI64($iter568);
           }
         }
         $output->writeListEnd();
@@ -17688,15 +17737,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size562 = 0;
-            $_etype565 = 0;
-            $xfer += $input->readListBegin($_etype565, $_size562);
-            for ($_i566 = 0; $_i566 < $_size562; ++$_i566)
+            $_size569 = 0;
+            $_etype572 = 0;
+            $xfer += $input->readListBegin($_etype572, $_size569);
+            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
             {
-              $elem567 = null;
-              $elem567 = new \metastore\Function();
-              $xfer += $elem567->read($input);
-              $this->functions []= $elem567;
+              $elem574 = null;
+              $elem574 = new \metastore\Function();
+              $xfer += $elem574->read($input);
+              $this->functions []= $elem574;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17724,9 +17773,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter568)
+          foreach ($this->functions as $iter575)
           {
-            $xfer += $iter568->write($output);
+            $xfer += $iter575->write($output);
           }
         }
         $output->writeListEnd();
@@ -17790,14 +17839,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size569 = 0;
-            $_etype572 = 0;
-            $xfer += $input->readListBegin($_etype572, $_size569);
-            for ($_i573 = 0; $_i573 < $_size569; ++$_i573)
+            $_size576 = 0;
+            $_etype579 = 0;
+            $xfer += $input->readListBegin($_etype579, $_size576);
+            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
             {
-              $elem574 = null;
-              $xfer += $input->readI32($elem574);
-              $this->values []= $elem574;
+              $elem581 = null;
+              $xfer += $input->readI32($elem581);
+              $this->values []= $elem581;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17825,9 +17874,9 @@ class ClientCapabilities {
       {
         $output->writeListBegin(TType::I32, count($this->values));
         {
-          foreach ($this->values as $iter575)
+          foreach ($this->values as $iter582)
           {
-            $xfer += $output->writeI32($iter575);
+            $xfer += $output->writeI32($iter582);
           }
         }
         $output->writeListEnd();
@@ -18127,14 +18176,14 @@ class GetTablesRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->tblNames = array();
-            $_size576 = 0;
-            $_etype579 = 0;
-            $xfer += $input->readListBegin($_etype579, $_size576);
-            for ($_i580 = 0; $_i580 < $_size576; ++$_i580)
+            $_size583 = 0;
+            $_etype586 = 0;
+            $xfer += $input->readListBegin($_etype586, $_size583);
+            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
             {
-              $elem581 = null;
-              $xfer += $input->readString($elem581);
-              $this->tblNames []= $elem581;
+              $elem588 = null;
+              $xfer += $input->readString($elem588);
+              $this->tblNames []= $elem588;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18175,9 +18224,9 @@ class GetTablesRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->tblNames));
         {
-          foreach ($this->tblNames as $iter582)
+          foreach ($this->tblNames as $iter589)
           {
-            $xfer += $output->writeString($iter582);
+            $xfer += $output->writeString($iter589);
           }
         }
         $output->writeListEnd();
@@ -18250,15 +18299,15 @@ class GetTablesResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tables = array();
-            $_size583 = 0;
-            $_etype586 = 0;
-            $xfer += $input->readListBegin($_etype586, $_size583);
-            for ($_i587 = 0; $_i587 < $_size583; ++$_i587)
+            $_size590 = 0;
+            $_etype593 = 0;
+            $xfer += $input->readListBegin($_etype593, $_size590);
+            for ($_i594 = 0; $_i594 < $_size590; ++$_i594)
             {
-              $elem588 = null;
-              $elem588 = new \metastore\Table();
-              $xfer += $elem588->read($input);
-              $this->tables []= $elem588;
+              $elem595 = null;
+              $elem595 = new \metastore\Table();
+              $xfer += $elem595->read($input);
+              $this->tables []= $elem595;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18286,9 +18335,9 @@ class GetTablesResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tables));
         {
-          foreach ($this->tables as $iter589)
+          foreach ($this->tables as $iter596)
           {
-            $xfer += $iter589->write($output);
+            $xfer += $iter596->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/bbd99ed6/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 bc51b75..86bbef3 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -11551,10 +11551,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype590, _size587) = iprot.readListBegin()
-          for _i591 in xrange(_size587):
-            _elem592 = iprot.readString()
-            self.success.append(_elem592)
+          (_etype597, _size594) = iprot.readListBegin()
+          for _i598 in xrange(_size594):
+            _elem599 = iprot.readString()
+            self.success.append(_elem599)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11577,8 +11577,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 iter593 in self.success:
-        oprot.writeString(iter593)
+      for iter600 in self.success:
+        oprot.writeString(iter600)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11683,10 +11683,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype597, _size594) = iprot.readListBegin()
-          for _i598 in xrange(_size594):
-            _elem599 = iprot.readString()
-            self.success.append(_elem599)
+          (_etype604, _size601) = iprot.readListBegin()
+          for _i605 in xrange(_size601):
+            _elem606 = iprot.readString()
+            self.success.append(_elem606)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11709,8 +11709,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 iter600 in self.success:
-        oprot.writeString(iter600)
+      for iter607 in self.success:
+        oprot.writeString(iter607)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12480,12 +12480,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype602, _vtype603, _size601 ) = iprot.readMapBegin()
-          for _i605 in xrange(_size601):
-            _key606 = iprot.readString()
-            _val607 = Type()
-            _val607.read(iprot)
-            self.success[_key606] = _val607
+          (_ktype609, _vtype610, _size608 ) = iprot.readMapBegin()
+          for _i612 in xrange(_size608):
+            _key613 = iprot.readString()
+            _val614 = Type()
+            _val614.read(iprot)
+            self.success[_key613] = _val614
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -12508,9 +12508,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 kiter608,viter609 in self.success.items():
-        oprot.writeString(kiter608)
-        viter609.write(oprot)
+      for kiter615,viter616 in self.success.items():
+        oprot.writeString(kiter615)
+        viter616.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -12653,11 +12653,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype613, _size610) = iprot.readListBegin()
-          for _i614 in xrange(_size610):
-            _elem615 = FieldSchema()
-            _elem615.read(iprot)
-            self.success.append(_elem615)
+          (_etype620, _size617) = iprot.readListBegin()
+          for _i621 in xrange(_size617):
+            _elem622 = FieldSchema()
+            _elem622.read(iprot)
+            self.success.append(_elem622)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12692,8 +12692,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 iter616 in self.success:
-        iter616.write(oprot)
+      for iter623 in self.success:
+        iter623.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12860,11 +12860,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype620, _size617) = iprot.readListBegin()
-          for _i621 in xrange(_size617):
-            _elem622 = FieldSchema()
-            _elem622.read(iprot)
-            self.success.append(_elem622)
+          (_etype627, _size624) = iprot.readListBegin()
+          for _i628 in xrange(_size624):
+            _elem629 = FieldSchema()
+            _elem629.read(iprot)
+            self.success.append(_elem629)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12899,8 +12899,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 iter623 in self.success:
-        iter623.write(oprot)
+      for iter630 in self.success:
+        iter630.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13053,11 +13053,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype627, _size624) = iprot.readListBegin()
-          for _i628 in xrange(_size624):
-            _elem629 = FieldSchema()
-            _elem629.read(iprot)
-            self.success.append(_elem629)
+          (_etype634, _size631) = iprot.readListBegin()
+          for _i635 in xrange(_size631):
+            _elem636 = FieldSchema()
+            _elem636.read(iprot)
+            self.success.append(_elem636)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13092,8 +13092,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 iter630 in self.success:
-        iter630.write(oprot)
+      for iter637 in self.success:
+        iter637.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13260,11 +13260,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype634, _size631) = iprot.readListBegin()
-          for _i635 in xrange(_size631):
-            _elem636 = FieldSchema()
-            _elem636.read(iprot)
-            self.success.append(_elem636)
+          (_etype641, _size638) = iprot.readListBegin()
+          for _i642 in xrange(_size638):
+            _elem643 = FieldSchema()
+            _elem643.read(iprot)
+            self.success.append(_elem643)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13299,8 +13299,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 iter637 in self.success:
-        iter637.write(oprot)
+      for iter644 in self.success:
+        iter644.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13741,22 +13741,22 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype641, _size638) = iprot.readListBegin()
-          for _i642 in xrange(_size638):
-            _elem643 = SQLPrimaryKey()
-            _elem643.read(iprot)
-            self.primaryKeys.append(_elem643)
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = SQLPrimaryKey()
+            _elem650.read(iprot)
+            self.primaryKeys.append(_elem650)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype647, _size644) = iprot.readListBegin()
-          for _i648 in xrange(_size644):
-            _elem649 = SQLForeignKey()
-            _elem649.read(iprot)
-            self.foreignKeys.append(_elem649)
+          (_etype654, _size651) = iprot.readListBegin()
+          for _i655 in xrange(_size651):
+            _elem656 = SQLForeignKey()
+            _elem656.read(iprot)
+            self.foreignKeys.append(_elem656)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13777,15 +13777,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 iter650 in self.primaryKeys:
-        iter650.write(oprot)
+      for iter657 in self.primaryKeys:
+        iter657.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 iter651 in self.foreignKeys:
-        iter651.write(oprot)
+      for iter658 in self.foreignKeys:
+        iter658.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14821,10 +14821,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype655, _size652) = iprot.readListBegin()
-          for _i656 in xrange(_size652):
-            _elem657 = iprot.readString()
-            self.success.append(_elem657)
+          (_etype662, _size659) = iprot.readListBegin()
+          for _i663 in xrange(_size659):
+            _elem664 = iprot.readString()
+            self.success.append(_elem664)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14847,8 +14847,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 iter658 in self.success:
-        oprot.writeString(iter658)
+      for iter665 in self.success:
+        oprot.writeString(iter665)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14998,10 +14998,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype662, _size659) = iprot.readListBegin()
-          for _i663 in xrange(_size659):
-            _elem664 = iprot.readString()
-            self.success.append(_elem664)
+          (_etype669, _size666) = iprot.readListBegin()
+          for _i670 in xrange(_size666):
+            _elem671 = iprot.readString()
+            self.success.append(_elem671)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15024,8 +15024,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter665 in self.success:
-        oprot.writeString(iter665)
+      for iter672 in self.success:
+        oprot.writeString(iter672)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15098,10 +15098,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype669, _size666) = iprot.readListBegin()
-          for _i670 in xrange(_size666):
-            _elem671 = iprot.readString()
-            self.tbl_types.append(_elem671)
+          (_etype676, _size673) = iprot.readListBegin()
+          for _i677 in xrange(_size673):
+            _elem678 = iprot.readString()
+            self.tbl_types.append(_elem678)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15126,8 +15126,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 iter672 in self.tbl_types:
-        oprot.writeString(iter672)
+      for iter679 in self.tbl_types:
+        oprot.writeString(iter679)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15183,11 +15183,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype676, _size673) = iprot.readListBegin()
-          for _i677 in xrange(_size673):
-            _elem678 = TableMeta()
-            _elem678.read(iprot)
-            self.success.append(_elem678)
+          (_etype683, _size680) = iprot.readListBegin()
+          for _i684 in xrange(_size680):
+            _elem685 = TableMeta()
+            _elem685.read(iprot)
+            self.success.append(_elem685)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15210,8 +15210,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 iter679 in self.success:
-        iter679.write(oprot)
+      for iter686 in self.success:
+        iter686.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15335,10 +15335,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype683, _size680) = iprot.readListBegin()
-          for _i684 in xrange(_size680):
-            _elem685 = iprot.readString()
-            self.success.append(_elem685)
+          (_etype690, _size687) = iprot.readListBegin()
+          for _i691 in xrange(_size687):
+            _elem692 = iprot.readString()
+            self.success.append(_elem692)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15361,8 +15361,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 iter686 in self.success:
-        oprot.writeString(iter686)
+      for iter693 in self.success:
+        oprot.writeString(iter693)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15598,10 +15598,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype690, _size687) = iprot.readListBegin()
-          for _i691 in xrange(_size687):
-            _elem692 = iprot.readString()
-            self.tbl_names.append(_elem692)
+          (_etype697, _size694) = iprot.readListBegin()
+          for _i698 in xrange(_size694):
+            _elem699 = iprot.readString()
+            self.tbl_names.append(_elem699)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15622,8 +15622,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 iter693 in self.tbl_names:
-        oprot.writeString(iter693)
+      for iter700 in self.tbl_names:
+        oprot.writeString(iter700)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15675,11 +15675,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype697, _size694) = iprot.readListBegin()
-          for _i698 in xrange(_size694):
-            _elem699 = Table()
-            _elem699.read(iprot)
-            self.success.append(_elem699)
+          (_etype704, _size701) = iprot.readListBegin()
+          for _i705 in xrange(_size701):
+            _elem706 = Table()
+            _elem706.read(iprot)
+            self.success.append(_elem706)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15696,8 +15696,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 iter700 in self.success:
-        iter700.write(oprot)
+      for iter707 in self.success:
+        iter707.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16180,10 +16180,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype704, _size701) = iprot.readListBegin()
-          for _i705 in xrange(_size701):
-            _elem706 = iprot.readString()
-            self.success.append(_elem706)
+          (_etype711, _size708) = iprot.readListBegin()
+          for _i712 in xrange(_size708):
+            _elem713 = iprot.readString()
+            self.success.append(_elem713)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16218,8 +16218,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 iter707 in self.success:
-        oprot.writeString(iter707)
+      for iter714 in self.success:
+        oprot.writeString(iter714)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17189,11 +17189,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype711, _size708) = iprot.readListBegin()
-          for _i712 in xrange(_size708):
-            _elem713 = Partition()
-            _elem713.read(iprot)
-            self.new_parts.append(_elem713)
+          (_etype718, _size715) = iprot.readListBegin()
+          for _i719 in xrange(_size715):
+            _elem720 = Partition()
+            _elem720.read(iprot)
+            self.new_parts.append(_elem720)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17210,8 +17210,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 iter714 in self.new_parts:
-        iter714.write(oprot)
+      for iter721 in self.new_parts:
+        iter721.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17369,11 +17369,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype718, _size715) = iprot.readListBegin()
-          for _i719 in xrange(_size715):
-            _elem720 = PartitionSpec()
-            _elem720.read(iprot)
-            self.new_parts.append(_elem720)
+          (_etype725, _size722) = iprot.readListBegin()
+          for _i726 in xrange(_size722):
+            _elem727 = PartitionSpec()
+            _elem727.read(iprot)
+            self.new_parts.append(_elem727)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17390,8 +17390,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 iter721 in self.new_parts:
-        iter721.write(oprot)
+      for iter728 in self.new_parts:
+        iter728.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17565,10 +17565,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype725, _size722) = iprot.readListBegin()
-          for _i726 in xrange(_size722):
-            _elem727 = iprot.readString()
-            self.part_vals.append(_elem727)
+          (_etype732, _size729) = iprot.readListBegin()
+          for _i733 in xrange(_size729):
+            _elem734 = iprot.readString()
+            self.part_vals.append(_elem734)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17593,8 +17593,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 iter728 in self.part_vals:
-        oprot.writeString(iter728)
+      for iter735 in self.part_vals:
+        oprot.writeString(iter735)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17947,10 +17947,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype732, _size729) = iprot.readListBegin()
-          for _i733 in xrange(_size729):
-            _elem734 = iprot.readString()
-            self.part_vals.append(_elem734)
+          (_etype739, _size736) = iprot.readListBegin()
+          for _i740 in xrange(_size736):
+            _elem741 = iprot.readString()
+            self.part_vals.append(_elem741)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17981,8 +17981,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 iter735 in self.part_vals:
-        oprot.writeString(iter735)
+      for iter742 in self.part_vals:
+        oprot.writeString(iter742)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -18577,10 +18577,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype739, _size736) = iprot.readListBegin()
-          for _i740 in xrange(_size736):
-            _elem741 = iprot.readString()
-            self.part_vals.append(_elem741)
+          (_etype746, _size743) = iprot.readListBegin()
+          for _i747 in xrange(_size743):
+            _elem748 = iprot.readString()
+            self.part_vals.append(_elem748)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18610,8 +18610,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 iter742 in self.part_vals:
-        oprot.writeString(iter742)
+      for iter749 in self.part_vals:
+        oprot.writeString(iter749)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -18784,10 +18784,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype746, _size743) = iprot.readListBegin()
-          for _i747 in xrange(_size743):
-            _elem748 = iprot.readString()
-            self.part_vals.append(_elem748)
+          (_etype753, _size750) = iprot.readListBegin()
+          for _i754 in xrange(_size750):
+            _elem755 = iprot.readString()
+            self.part_vals.append(_elem755)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18823,8 +18823,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 iter749 in self.part_vals:
-        oprot.writeString(iter749)
+      for iter756 in self.part_vals:
+        oprot.writeString(iter756)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -19561,10 +19561,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype753, _size750) = iprot.readListBegin()
-          for _i754 in xrange(_size750):
-            _elem755 = iprot.readString()
-            self.part_vals.append(_elem755)
+          (_etype760, _size757) = iprot.readListBegin()
+          for _i761 in xrange(_size757):
+            _elem762 = iprot.readString()
+            self.part_vals.append(_elem762)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19589,8 +19589,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 iter756 in self.part_vals:
-        oprot.writeString(iter756)
+      for iter763 in self.part_vals:
+        oprot.writeString(iter763)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19749,11 +19749,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype758, _vtype759, _size757 ) = iprot.readMapBegin()
-          for _i761 in xrange(_size757):
-            _key762 = iprot.readString()
-            _val763 = iprot.readString()
-            self.partitionSpecs[_key762] = _val763
+          (_ktype765, _vtype766, _size764 ) = iprot.readMapBegin()
+          for _i768 in xrange(_size764):
+            _key769 = iprot.readString()
+            _val770 = iprot.readString()
+            self.partitionSpecs[_key769] = _val770
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -19790,9 +19790,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 kiter764,viter765 in self.partitionSpecs.items():
-        oprot.writeString(kiter764)
-        oprot.writeString(viter765)
+      for kiter771,viter772 in self.partitionSpecs.items():
+        oprot.writeString(kiter771)
+        oprot.writeString(viter772)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -19997,11 +19997,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype767, _vtype768, _size766 ) = iprot.readMapBegin()
-          for _i770 in xrange(_size766):
-            _key771 = iprot.readString()
-            _val772 = iprot.readString()
-            self.partitionSpecs[_key771] = _val772
+          (_ktype774, _vtype775, _size773 ) = iprot.readMapBegin()
+          for _i777 in xrange(_size773):
+            _key778 = iprot.readString()
+            _val779 = iprot.readString()
+            self.partitionSpecs[_key778] = _val779
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -20038,9 +20038,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 kiter773,viter774 in self.partitionSpecs.items():
-        oprot.writeString(kiter773)
-        oprot.writeString(viter774)
+      for kiter780,viter781 in self.partitionSpecs.items():
+        oprot.writeString(kiter780)
+        oprot.writeString(viter781)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -20123,11 +20123,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype778, _size775) = iprot.readListBegin()
-          for _i779 in xrange(_size775):
-            _elem780 = Partition()
-            _elem780.read(iprot)
-            self.success.append(_elem780)
+          (_etype785, _size782) = iprot.readListBegin()
+          for _i786 in xrange(_size782):
+            _elem787 = Partition()
+            _elem787.read(iprot)
+            self.success.append(_elem787)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20168,8 +20168,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 iter781 in self.success:
-        iter781.write(oprot)
+      for iter788 in self.success:
+        iter788.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20263,10 +20263,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype785, _size782) = iprot.readListBegin()
-          for _i786 in xrange(_size782):
-            _elem787 = iprot.readString()
-            self.part_vals.append(_elem787)
+          (_etype792, _size789) = iprot.readListBegin()
+          for _i793 in xrange(_size789):
+            _elem794 = iprot.readString()
+            self.part_vals.append(_elem794)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20278,10 +20278,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype791, _size788) = iprot.readListBegin()
-          for _i792 in xrange(_size788):
-            _elem793 = iprot.readString()
-            self.group_names.append(_elem793)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = iprot.readString()
+            self.group_names.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20306,8 +20306,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 iter794 in self.part_vals:
-        oprot.writeString(iter794)
+      for iter801 in self.part_vals:
+        oprot.writeString(iter801)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -20317,8 +20317,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 iter795 in self.group_names:
-        oprot.writeString(iter795)
+      for iter802 in self.group_names:
+        oprot.writeString(iter802)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20747,11 +20747,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype799, _size796) = iprot.readListBegin()
-          for _i800 in xrange(_size796):
-            _elem801 = Partition()
-            _elem801.read(iprot)
-            self.success.append(_elem801)
+          (_etype806, _size803) = iprot.readListBegin()
+          for _i807 in xrange(_size803):
+            _elem808 = Partition()
+            _elem808.read(iprot)
+            self.success.append(_elem808)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20780,8 +20780,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 iter802 in self.success:
-        iter802.write(oprot)
+      for iter809 in self.success:
+        iter809.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20875,10 +20875,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype806, _size803) = iprot.readListBegin()
-          for _i807 in xrange(_size803):
-            _elem808 = iprot.readString()
-            self.group_names.append(_elem808)
+          (_etype813, _size810) = iprot.readListBegin()
+          for _i814 in xrange(_size810):
+            _elem815 = iprot.readString()
+            self.group_names.append(_elem815)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20911,8 +20911,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 iter809 in self.group_names:
-        oprot.writeString(iter809)
+      for iter816 in self.group_names:
+        oprot.writeString(iter816)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20973,11 +20973,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Partition()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = Partition()
+            _elem822.read(iprot)
+            self.success.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21006,8 +21006,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 iter816 in self.success:
-        iter816.write(oprot)
+      for iter823 in self.success:
+        iter823.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21165,11 +21165,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = PartitionSpec()
-            _elem822.read(iprot)
-            self.success.append(_elem822)
+          (_etype827, _size824) = iprot.readListBegin()
+          for _i828 in xrange(_size824):
+            _elem829 = PartitionSpec()
+            _elem829.read(iprot)
+            self.success.append(_elem829)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21198,8 +21198,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 iter823 in self.success:
-        iter823.write(oprot)
+      for iter830 in self.success:
+        iter830.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21354,10 +21354,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype827, _size824) = iprot.readListBegin()
-          for _i828 in xrange(_size824):
-            _elem829 = iprot.readString()
-            self.success.append(_elem829)
+          (_etype834, _size831) = iprot.readListBegin()
+          for _i835 in xrange(_size831):
+            _elem836 = iprot.readString()
+            self.success.append(_elem836)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21380,8 +21380,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 iter830 in self.success:
-        oprot.writeString(iter830)
+      for iter837 in self.success:
+        oprot.writeString(iter837)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -21457,10 +21457,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = iprot.readString()
-            self.part_vals.append(_elem836)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = iprot.readString()
+            self.part_vals.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21490,8 +21490,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 iter837 in self.part_vals:
-        oprot.writeString(iter837)
+      for iter844 in self.part_vals:
+        oprot.writeString(iter844)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21555,11 +21555,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = Partition()
-            _elem843.read(iprot)
-            self.success.append(_elem843)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = Partition()
+            _elem850.read(iprot)
+            self.success.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21588,8 +21588,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 iter844 in self.success:
-        iter844.write(oprot)
+      for iter851 in self.success:
+        iter851.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21676,10 +21676,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.part_vals.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = iprot.readString()
+            self.part_vals.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21696,10 +21696,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype854, _size851) = iprot.readListBegin()
-          for _i855 in xrange(_size851):
-            _elem856 = iprot.readString()
-            self.group_names.append(_elem856)
+          (_etype861, _size858) = iprot.readListBegin()
+          for _i862 in xrange(_size858):
+            _elem863 = iprot.readString()
+            self.group_names.append(_elem863)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21724,8 +21724,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 iter857 in self.part_vals:
-        oprot.writeString(iter857)
+      for iter864 in self.part_vals:
+        oprot.writeString(iter864)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -21739,8 +21739,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 iter858 in self.group_names:
-        oprot.writeString(iter858)
+      for iter865 in self.group_names:
+        oprot.writeString(iter865)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21802,11 +21802,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = Partition()
-            _elem864.read(iprot)
-            self.success.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = Partition()
+            _elem871.read(iprot)
+            self.success.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21835,8 +21835,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 iter865 in self.success:
-        iter865.write(oprot)
+      for iter872 in self.success:
+        iter872.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21917,10 +21917,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.part_vals.append(_elem871)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = iprot.readString()
+            self.part_vals.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21950,8 +21950,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 iter872 in self.part_vals:
-        oprot.writeString(iter872)
+      for iter879 in self.part_vals:
+        oprot.writeString(iter879)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -22015,10 +22015,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = iprot.readString()
-            self.success.append(_elem878)
+          (_etype883, _size880) = iprot.readListBegin()
+          for _i884 in xrange(_size880):
+            _elem885 = iprot.readString()
+            self.success.append(_elem885)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22047,8 +22047,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 iter879 in self.success:
-        oprot.writeString(iter879)
+      for iter886 in self.success:
+        oprot.writeString(iter886)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22219,11 +22219,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype883, _size880) = iprot.readListBegin()
-          for _i884 in xrange(_size880):
-            _elem885 = Partition()
-            _elem885.read(iprot)
-            self.success.append(_elem885)
+          (_etype890, _size887) = iprot.readListBegin()
+          for _i891 in xrange(_size887):
+            _elem892 = Partition()
+            _elem892.read(iprot)
+            self.success.append(_elem892)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22252,8 +22252,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 iter886 in self.success:
-        iter886.write(oprot)
+      for iter893 in self.success:
+        iter893.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22424,11 +22424,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype890, _size887) = iprot.readListBegin()
-          for _i891 in xrange(_size887):
-            _elem892 = PartitionSpec()
-            _elem892.read(iprot)
-            self.success.append(_elem892)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = PartitionSpec()
+            _elem899.read(iprot)
+            self.success.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22457,8 +22457,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 iter893 in self.success:
-        iter893.write(oprot)
+      for iter900 in self.success:
+        iter900.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22878,10 +22878,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype897, _size894) = iprot.readListBegin()
-          for _i898 in xrange(_size894):
-            _elem899 = iprot.readString()
-            self.names.append(_elem899)
+          (_etype904, _size901) = iprot.readListBegin()
+          for _i905 in xrange(_size901):
+            _elem906 = iprot.readString()
+            self.names.append(_elem906)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22906,8 +22906,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 iter900 in self.names:
-        oprot.writeString(iter900)
+      for iter907 in self.names:
+        oprot.writeString(iter907)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22966,11 +22966,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype904, _size901) = iprot.readListBegin()
-          for _i905 in xrange(_size901):
-            _elem906 = Partition()
-            _elem906.read(iprot)
-            self.success.append(_elem906)
+          (_etype911, _size908) = iprot.readListBegin()
+          for _i912 in xrange(_size908):
+            _elem913 = Partition()
+            _elem913.read(iprot)
+            self.success.append(_elem913)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22999,8 +22999,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 iter907 in self.success:
-        iter907.write(oprot)
+      for iter914 in self.success:
+        iter914.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23250,11 +23250,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = Partition()
-            _elem913.read(iprot)
-            self.new_parts.append(_elem913)
+          (_etype918, _size915) = iprot.readListBegin()
+          for _i919 in xrange(_size915):
+            _elem920 = Partition()
+            _elem920.read(iprot)
+            self.new_parts.append(_elem920)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23279,8 +23279,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 iter914 in self.new_parts:
-        iter914.write(oprot)
+      for iter921 in self.new_parts:
+        iter921.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23433,11 +23433,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype918, _size915) = iprot.readListBegin()
-          for _i919 in xrange(_size915):
-            _elem920 = Partition()
-            _elem920.read(iprot)
-            self.new_parts.append(_elem920)
+          (_etype925, _size922) = iprot.readListBegin()
+          for _i926 in xrange(_size922):
+            _elem927 = Partition()
+            _elem927.read(iprot)
+            self.new_parts.append(_elem927)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23468,8 +23468,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 iter921 in self.new_parts:
-        iter921.write(oprot)
+      for iter928 in self.new_parts:
+        iter928.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -23813,10 +23813,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype925, _size922) = iprot.readListBegin()
-          for _i926 in xrange(_size922):
-            _elem927 = iprot.readString()
-            self.part_vals.append(_elem927)
+          (_etype932, _size929) = iprot.readListBegin()
+          for _i933 in xrange(_size929):
+            _elem934 = iprot.readString()
+            self.part_vals.append(_elem934)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23847,8 +23847,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 iter928 in self.part_vals:
-        oprot.writeString(iter928)
+      for iter935 in self.part_vals:
+        oprot.writeString(iter935)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -23990,10 +23990,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype932, _size929) = iprot.readListBegin()
-          for _i933 in xrange(_size929):
-            _elem934 = iprot.readString()
-            self.part_vals.append(_elem934)
+          (_etype939, _size936) = iprot.readListBegin()
+          for _i940 in xrange(_size936):
+            _elem941 = iprot.readString()
+            self.part_vals.append(_elem941)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24015,8 +24015,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 iter935 in self.part_vals:
-        oprot.writeString(iter935)
+      for iter942 in self.part_vals:
+        oprot.writeString(iter942)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -24374,10 +24374,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype939, _size936) = iprot.readListBegin()
-          for _i940 in xrange(_size936):
-            _elem941 = iprot.readString()
-            self.success.append(_elem941)
+          (_etype946, _size943) = iprot.readListBegin()
+          for _i947 in xrange(_size943):
+            _elem948 = iprot.readString()
+            self.success.append(_elem948)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24400,8 +24400,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 iter942 in self.success:
-        oprot.writeString(iter942)
+      for iter949 in self.success:
+        oprot.writeString(iter949)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24525,11 +24525,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype944, _vtype945, _size943 ) = iprot.readMapBegin()
-          for _i947 in xrange(_size943):
-            _key948 = iprot.readString()
-            _val949 = iprot.readString()
-            self.success[_key948] = _val949
+          (_ktype951, _vtype952, _size950 ) = iprot.readMapBegin()
+          for _i954 in xrange(_size950):
+            _key955 = iprot.readString()
+            _val956 = iprot.readString()
+            self.success[_key955] = _val956
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24552,9 +24552,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 kiter950,viter951 in self.success.items():
-        oprot.writeString(kiter950)
-        oprot.writeString(viter951)
+      for kiter957,viter958 in self.success.items():
+        oprot.writeString(kiter957)
+        oprot.writeString(viter958)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24630,11 +24630,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype953, _vtype954, _size952 ) = iprot.readMapBegin()
-          for _i956 in xrange(_size952):
-            _key957 = iprot.readString()
-            _val958 = iprot.readString()
-            self.part_vals[_key957] = _val958
+          (_ktype960, _vtype961, _size959 ) = iprot.readMapBegin()
+          for _i963 in xrange(_size959):
+            _key964 = iprot.readString()
+            _val965 = iprot.readString()
+            self.part_vals[_key964] = _val965
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24664,9 +24664,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 kiter959,viter960 in self.part_vals.items():
-        oprot.writeString(kiter959)
-        oprot.writeString(viter960)
+      for kiter966,viter967 in self.part_vals.items():
+        oprot.writeString(kiter966)
+        oprot.writeString(viter967)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -24880,11 +24880,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype962, _vtype963, _size961 ) = iprot.readMapBegin()
-          for _i965 in xrange(_size961):
-            _key966 = iprot.readString()
-            _val967 = iprot.readString()
-            self.part_vals[_key966] = _val967
+          (_ktype969, _vtype970, _size968 ) = iprot.readMapBegin()
+          for _i972 in xrange(_size968):
+            _key973 = iprot.readString()
+            _val974 = iprot.readString()
+            self.part_vals[_key973] = _val974
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -24914,9 +24914,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 kiter968,viter969 in self.part_vals.items():
-        oprot.writeString(kiter968)
-        oprot.writeString(viter969)
+      for kiter975,viter976 in self.part_vals.items():
+        oprot.writeString(kiter975)
+        oprot.writeString(viter976)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -25971,11 +25971,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype973, _size970) = iprot.readListBegin()
-          for _i974 in xrange(_size970):
-            _elem975 = Index()
-            _elem975.read(iprot)
-            self.success.append(_elem975)
+          (_etype980, _size977) = iprot.readListBegin()
+          for _i981 in xrange(_size977):
+            _elem982 = Index()
+            _elem982.read(iprot)
+            self.success.append(_elem982)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26004,8 +26004,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 iter976 in self.success:
-        iter976.write(oprot)
+      for iter983 in self.success:
+        iter983.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26160,10 +26160,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype980, _size977) = iprot.readListBegin()
-          for _i981 in xrange(_size977):
-            _elem982 = iprot.readString()
-            self.success.append(_elem982)
+          (_etype987, _size984) = iprot.readListBegin()
+          for _i988 in xrange(_size984):
+            _elem989 = iprot.readString()
+            self.success.append(_elem989)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26186,8 +26186,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 iter983 in self.success:
-        oprot.writeString(iter983)
+      for iter990 in self.success:
+        oprot.writeString(iter990)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -29053,10 +29053,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype987, _size984) = iprot.readListBegin()
-          for _i988 in xrange(_size984):
-            _elem989 = iprot.readString()
-            self.success.append(_elem989)
+          (_etype994, _size991) = iprot.readListBegin()
+          for _i995 in xrange(_size991):
+            _elem996 = iprot.readString()
+            self.success.append(_elem996)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29079,8 +29079,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 iter990 in self.success:
-        oprot.writeString(iter990)
+      for iter997 in self.success:
+        oprot.writeString(iter997)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29768,10 +29768,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype994, _size991) = iprot.readListBegin()
-          for _i995 in xrange(_size991):
-            _elem996 = iprot.readString()
-            self.success.append(_elem996)
+          (_etype1001, _size998) = iprot.readListBegin()
+          for _i1002 in xrange(_size998):
+            _elem1003 = iprot.readString()
+            self.success.append(_elem1003)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29794,8 +29794,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 iter997 in self.success:
-        oprot.writeString(iter997)
+      for iter1004 in self.success:
+        oprot.writeString(iter1004)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30309,11 +30309,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1001, _size998) = iprot.readListBegin()
-          for _i1002 in xrange(_size998):
-            _elem1003 = Role()
-            _elem1003.read(iprot)
-            self.success.append(_elem1003)
+          (_etype1008, _size1005) = iprot.readListBegin()
+          for _i1009 in xrange(_size1005):
+            _elem1010 = Role()
+            _elem1010.read(iprot)
+            self.success.append(_elem1010)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30336,8 +30336,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 iter1004 in self.success:
-        iter1004.write(oprot)
+      for iter1011 in self.success:
+        iter1011.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30846,10 +30846,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1008, _size1005) = iprot.readListBegin()
-          for _i1009 in xrange(_size1005):
-            _elem1010 = iprot.readString()
-            self.group_names.append(_elem1010)
+          (_etype1015, _size1012) = iprot.readListBegin()
+          for _i1016 in xrange(_size1012):
+            _elem1017 = iprot.readString()
+            self.group_names.append(_elem1017)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30874,8 +30874,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 iter1011 in self.group_names:
-        oprot.writeString(iter1011)
+      for iter1018 in self.group_names:
+        oprot.writeString(iter1018)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -31102,11 +31102,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1015, _size1012) = iprot.readListBegin()
-          for _i1016 in xrange(_size1012):
-            _elem1017 = HiveObjectPrivilege()
-            _elem1017.read(iprot)
-            self.success.append(_elem1017)
+          (_etype1022, _size1019) = iprot.readListBegin()
+          for _i1023 in xrange(_size1019):
+            _elem1024 = HiveObjectPrivilege()
+            _elem1024.read(iprot)
+            self.success.append(_elem1024)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31129,8 +31129,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 iter1018 in self.success:
-        iter1018.write(oprot)
+      for iter1025 in self.success:
+        iter1025.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31628,10 +31628,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1022, _size1019) = iprot.readListBegin()
-          for _i1023 in xrange(_size1019):
-            _elem1024 = iprot.readString()
-            self.group_names.append(_elem1024)
+          (_etype1029, _size1026) = iprot.readListBegin()
+          for _i1030 in xrange(_size1026):
+            _elem1031 = iprot.readString()
+            self.group_names.append(_elem1031)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31652,8 +31652,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 iter1025 in self.group_names:
-        oprot.writeString(iter1025)
+      for iter1032 in self.group_names:
+        oprot.writeString(iter1032)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -31708,10 +31708,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1029, _size1026) = iprot.readListBegin()
-          for _i1030 in xrange(_size1026):
-            _elem1031 = iprot.readString()
-            self.success.append(_elem1031)
+          (_etype1036, _size1033) = iprot.readListBegin()
+          for _i1037 in xrange(_size1033):
+            _elem1038 = iprot.readString()
+            self.success.append(_elem1038)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31734,8 +31734,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 iter1032 in self.success:
-        oprot.writeString(iter1032)
+      for iter1039 in self.success:
+        oprot.writeString(iter1039)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -32667,10 +32667,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1036, _size1033) = iprot.readListBegin()
-          for _i1037 in xrange(_size1033):
-            _elem1038 = iprot.readString()
-            self.success.append(_elem1038)
+          (_etype1043, _size1040) = iprot.readListBegin()
+          for _i1044 in xrange(_size1040):
+            _elem1045 = iprot.readString()
+            self.success.append(_elem1045)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -32687,8 +32687,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 iter1039 in self.success:
-        oprot.writeString(iter1039)
+      for iter1046 in self.success:
+        oprot.writeString(iter1046)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -33215,10 +33215,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1043, _size1040) = iprot.readListBegin()
-          for _i1044 in xrange(_size1040):
-            _elem1045 = iprot.readString()
-            self.success.append(_elem1045)
+          (_etype1050, _size1047) = iprot.readListBegin()
+          for _i1051 in xrange(_size1047):
+            _elem1052 = iprot.readString()
+            self.success.append(_elem1052)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -33235,8 +33235,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 iter1046 in self.success:
-        oprot.writeString(iter1046)
+      for iter1053 in self.success:
+        oprot.writeString(iter1053)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()