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

[1/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Repository: hive
Updated Branches:
  refs/heads/branch-1 f1950fc8a -> aecb0c02e


http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 596fdf5..c59fa3e 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -7483,6 +7483,81 @@ class AbortTxnRequest:
   def __ne__(self, other):
     return not (self == other)
 
+class AbortTxnsRequest:
+  """
+  Attributes:
+   - txn_ids
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'txn_ids', (TType.I64,None), None, ), # 1
+  )
+
+  def __init__(self, txn_ids=None,):
+    self.txn_ids = txn_ids
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.txn_ids = []
+          (_etype402, _size399) = iprot.readListBegin()
+          for _i403 in xrange(_size399):
+            _elem404 = iprot.readI64()
+            self.txn_ids.append(_elem404)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('AbortTxnsRequest')
+    if self.txn_ids is not None:
+      oprot.writeFieldBegin('txn_ids', TType.LIST, 1)
+      oprot.writeListBegin(TType.I64, len(self.txn_ids))
+      for iter405 in self.txn_ids:
+        oprot.writeI64(iter405)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.txn_ids is None:
+      raise TProtocol.TProtocolException(message='Required field txn_ids is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.txn_ids)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class CommitTxnRequest:
   """
   Attributes:
@@ -7711,11 +7786,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype402, _size399) = iprot.readListBegin()
-          for _i403 in xrange(_size399):
-            _elem404 = LockComponent()
-            _elem404.read(iprot)
-            self.component.append(_elem404)
+          (_etype409, _size406) = iprot.readListBegin()
+          for _i410 in xrange(_size406):
+            _elem411 = LockComponent()
+            _elem411.read(iprot)
+            self.component.append(_elem411)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7752,8 +7827,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter405 in self.component:
-        iter405.write(oprot)
+      for iter412 in self.component:
+        iter412.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -8451,11 +8526,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype409, _size406) = iprot.readListBegin()
-          for _i410 in xrange(_size406):
-            _elem411 = ShowLocksResponseElement()
-            _elem411.read(iprot)
-            self.locks.append(_elem411)
+          (_etype416, _size413) = iprot.readListBegin()
+          for _i417 in xrange(_size413):
+            _elem418 = ShowLocksResponseElement()
+            _elem418.read(iprot)
+            self.locks.append(_elem418)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -8472,8 +8547,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter412 in self.locks:
-        iter412.write(oprot)
+      for iter419 in self.locks:
+        iter419.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -8688,20 +8763,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype416, _size413) = iprot.readSetBegin()
-          for _i417 in xrange(_size413):
-            _elem418 = iprot.readI64()
-            self.aborted.add(_elem418)
+          (_etype423, _size420) = iprot.readSetBegin()
+          for _i424 in xrange(_size420):
+            _elem425 = iprot.readI64()
+            self.aborted.add(_elem425)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype422, _size419) = iprot.readSetBegin()
-          for _i423 in xrange(_size419):
-            _elem424 = iprot.readI64()
-            self.nosuch.add(_elem424)
+          (_etype429, _size426) = iprot.readSetBegin()
+          for _i430 in xrange(_size426):
+            _elem431 = iprot.readI64()
+            self.nosuch.add(_elem431)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -8718,15 +8793,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter425 in self.aborted:
-        oprot.writeI64(iter425)
+      for iter432 in self.aborted:
+        oprot.writeI64(iter432)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter426 in self.nosuch:
-        oprot.writeI64(iter426)
+      for iter433 in self.nosuch:
+        oprot.writeI64(iter433)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9168,11 +9243,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype430, _size427) = iprot.readListBegin()
-          for _i431 in xrange(_size427):
-            _elem432 = ShowCompactResponseElement()
-            _elem432.read(iprot)
-            self.compacts.append(_elem432)
+          (_etype437, _size434) = iprot.readListBegin()
+          for _i438 in xrange(_size434):
+            _elem439 = ShowCompactResponseElement()
+            _elem439.read(iprot)
+            self.compacts.append(_elem439)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9189,8 +9264,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter433 in self.compacts:
-        iter433.write(oprot)
+      for iter440 in self.compacts:
+        iter440.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9268,10 +9343,10 @@ class AddDynamicPartitions:
       elif fid == 4:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype437, _size434) = iprot.readListBegin()
-          for _i438 in xrange(_size434):
-            _elem439 = iprot.readString()
-            self.partitionnames.append(_elem439)
+          (_etype444, _size441) = iprot.readListBegin()
+          for _i445 in xrange(_size441):
+            _elem446 = iprot.readString()
+            self.partitionnames.append(_elem446)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9300,8 +9375,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 4)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter440 in self.partitionnames:
-        oprot.writeString(iter440)
+      for iter447 in self.partitionnames:
+        oprot.writeString(iter447)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9582,11 +9657,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype444, _size441) = iprot.readListBegin()
-          for _i445 in xrange(_size441):
-            _elem446 = NotificationEvent()
-            _elem446.read(iprot)
-            self.events.append(_elem446)
+          (_etype451, _size448) = iprot.readListBegin()
+          for _i452 in xrange(_size448):
+            _elem453 = NotificationEvent()
+            _elem453.read(iprot)
+            self.events.append(_elem453)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9603,8 +9678,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter447 in self.events:
-        iter447.write(oprot)
+      for iter454 in self.events:
+        iter454.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9725,10 +9800,10 @@ class InsertEventRequestData:
       if fid == 1:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype451, _size448) = iprot.readListBegin()
-          for _i452 in xrange(_size448):
-            _elem453 = iprot.readString()
-            self.filesAdded.append(_elem453)
+          (_etype458, _size455) = iprot.readListBegin()
+          for _i459 in xrange(_size455):
+            _elem460 = iprot.readString()
+            self.filesAdded.append(_elem460)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9745,8 +9820,8 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter454 in self.filesAdded:
-        oprot.writeString(iter454)
+      for iter461 in self.filesAdded:
+        oprot.writeString(iter461)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -9899,10 +9974,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype458, _size455) = iprot.readListBegin()
-          for _i459 in xrange(_size455):
-            _elem460 = iprot.readString()
-            self.partitionVals.append(_elem460)
+          (_etype465, _size462) = iprot.readListBegin()
+          for _i466 in xrange(_size462):
+            _elem467 = iprot.readString()
+            self.partitionVals.append(_elem467)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9935,8 +10010,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter461 in self.partitionVals:
-        oprot.writeString(iter461)
+      for iter468 in self.partitionVals:
+        oprot.writeString(iter468)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -10042,11 +10117,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype465, _size462) = iprot.readListBegin()
-          for _i466 in xrange(_size462):
-            _elem467 = Function()
-            _elem467.read(iprot)
-            self.functions.append(_elem467)
+          (_etype472, _size469) = iprot.readListBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = Function()
+            _elem474.read(iprot)
+            self.functions.append(_elem474)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10063,8 +10138,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter468 in self.functions:
-        iter468.write(oprot)
+      for iter475 in self.functions:
+        iter475.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 0028f48..2874308 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -1680,6 +1680,23 @@ class AbortTxnRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class AbortTxnsRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TXN_IDS = 1
+
+  FIELDS = {
+    TXN_IDS => {:type => ::Thrift::Types::LIST, :name => 'txn_ids', :element => {:type => ::Thrift::Types::I64}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txn_ids is unset!') unless @txn_ids
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class CommitTxnRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   TXNID = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 45f43f6..3a4974b 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -1905,6 +1905,21 @@ module ThriftHiveMetastore
       return
     end
 
+    def abort_txns(rqst)
+      send_abort_txns(rqst)
+      recv_abort_txns()
+    end
+
+    def send_abort_txns(rqst)
+      send_message('abort_txns', Abort_txns_args, :rqst => rqst)
+    end
+
+    def recv_abort_txns()
+      result = receive_message(Abort_txns_result)
+      raise result.o1 unless result.o1.nil?
+      return
+    end
+
     def commit_txn(rqst)
       send_commit_txn(rqst)
       recv_commit_txn()
@@ -3586,6 +3601,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'abort_txn', seqid)
     end
 
+    def process_abort_txns(seqid, iprot, oprot)
+      args = read_args(iprot, Abort_txns_args)
+      result = Abort_txns_result.new()
+      begin
+        @handler.abort_txns(args.rqst)
+      rescue ::NoSuchTxnException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'abort_txns', seqid)
+    end
+
     def process_commit_txn(seqid, iprot, oprot)
       args = read_args(iprot, Commit_txn_args)
       result = Commit_txn_result.new()
@@ -8065,6 +8091,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Abort_txns_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::AbortTxnsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Abort_txns_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchTxnException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Commit_txn_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     RQST = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 73422c8..7a2333d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
@@ -5619,6 +5620,11 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
 
     @Override
+    public void abort_txns(AbortTxnsRequest rqst) throws NoSuchTxnException, TException {
+      getTxnHandler().abortTxns(rqst);
+    }
+
+    @Override
     public void commit_txn(CommitTxnRequest rqst)
         throws NoSuchTxnException, TxnAbortedException, TException {
       getTxnHandler().commitTxn(rqst);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 50bf43c..6bef3f5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.conf.HiveConfUtil;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
@@ -1889,6 +1890,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   }
 
   @Override
+  public void abortTxns(List<Long> txnids) throws NoSuchTxnException, TException {
+    client.abort_txns(new AbortTxnsRequest(txnids));
+  }
+
+  @Override
   public LockResponse lock(LockRequest request)
       throws NoSuchTxnException, TxnAbortedException, TException {
     return client.lock(request);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 2de2cdb..da3d7bc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1240,6 +1240,12 @@ public interface IMetaStoreClient {
       throws NoSuchTxnException, TxnAbortedException, TException;
 
   /**
+   * Abort a list of transactions. This is for use by "ABORT TRANSACTIONS" in the grammar.
+   * @throws TException
+   */
+  void abortTxns(List<Long> txnids) throws TException;
+
+  /**
    * Show the list of currently open transactions.  This is for use by "show transactions" in the
    * grammar, not for applications that want to find a list of current transactions to work with.
    * Those wishing the latter should call {@link #getValidTxns()}.

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index ec60fa5..b1d330d 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -488,7 +488,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
       try {
         lockInternal();
         dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
-        if (abortTxns(dbConn, Collections.singletonList(txnid)) != 1) {
+        if (abortTxns(dbConn, Collections.singletonList(txnid), true) != 1) {
           LOG.debug("Going to rollback");
           dbConn.rollback();
           throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
@@ -511,6 +511,35 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   }
 
+  public void abortTxns(AbortTxnsRequest rqst) throws NoSuchTxnException, MetaException {
+    List<Long> txnids = rqst.getTxn_ids();
+    try {
+      Connection dbConn = null;
+      try {
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        int numAborted = abortTxns(dbConn, txnids, false);
+        if (numAborted != txnids.size()) {
+          LOG.warn("Abort Transactions command only abort " + numAborted + " out of " +
+              txnids.size() + " transactions. It's possible that the other " +
+              (txnids.size() - numAborted) +
+              " transactions have been aborted or committed, or the transaction ids are invalid.");
+        }
+        LOG.debug("Going to commit");
+        dbConn.commit();
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(dbConn, e, "abortTxns(" + rqst + ")");
+        throw new MetaException("Unable to update transaction database "
+            + StringUtils.stringifyException(e));
+      } finally {
+        closeDbConn(dbConn);
+      }
+    } catch (RetryException e) {
+      abortTxns(rqst);
+    }
+  }
+
   /**
    * Concurrency/isolation notes:
    * This is mutexed with {@link #openTxns(OpenTxnRequest)} and other {@link #commitTxn(CommitTxnRequest)}
@@ -629,7 +658,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             dbConn.rollback(undoWriteSetForCurrentTxn);
             LOG.info(msg);
             //todo: should make abortTxns() write something into TXNS.TXN_META_INFO about this
-            if(abortTxns(dbConn, Collections.singletonList(txnid)) != 1) {
+            if(abortTxns(dbConn, Collections.singletonList(txnid), true) != 1) {
               throw new IllegalStateException(msg + " FAILED!");
             }
             dbConn.commit();
@@ -2078,8 +2107,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
   }
 
-  private int abortTxns(Connection dbConn, List<Long> txnids) throws SQLException {
-    return abortTxns(dbConn, txnids, -1);
+  private int abortTxns(Connection dbConn, List<Long> txnids, boolean isStrict) throws SQLException {
+    return abortTxns(dbConn, txnids, -1, isStrict);
   }
   /**
    * TODO: expose this as an operation to client.  Useful for streaming API to abort all remaining
@@ -2091,10 +2120,15 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
    * @param txnids list of transactions to abort
    * @param max_heartbeat value used by {@link #performTimeOuts()} to ensure this doesn't Abort txn which were
    *                      hearbetated after #performTimeOuts() select and this operation.
+   * @param isStrict true for strict mode, false for best-effort mode.
+   *                 In strict mode, if all txns are not successfully aborted, then the count of
+   *                 updated ones will be returned and the caller will roll back.
+   *                 In best-effort mode, we will ignore that fact and continue deleting the locks.
    * @return Number of aborted transactions
    * @throws SQLException
    */
-  private int abortTxns(Connection dbConn, List<Long> txnids, long max_heartbeat) throws SQLException {
+  private int abortTxns(Connection dbConn, List<Long> txnids, long max_heartbeat, boolean isStrict)
+      throws SQLException {
     Statement stmt = null;
     int updateCnt = 0;
     if (txnids.isEmpty()) {
@@ -2121,16 +2155,17 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
       for (String query : queries) {
         LOG.debug("Going to execute update <" + query + ">");
-        updateCnt = stmt.executeUpdate(query);
-        if (updateCnt < txnids.size()) {
-          /**
-           * have to bail in this case since we don't know which transactions were not Aborted and
-           * thus don't know which locks to delete
-           * This may happen due to a race between {@link #heartbeat(HeartbeatRequest)}  operation and
-           * {@link #performTimeOuts()}
-           */
-          return updateCnt;
-        }
+        updateCnt += stmt.executeUpdate(query);
+      }
+
+      if (updateCnt < txnids.size() && isStrict) {
+        /**
+         * have to bail in this case since we don't know which transactions were not Aborted and
+         * thus don't know which locks to delete
+         * This may happen due to a race between {@link #heartbeat(HeartbeatRequest)}  operation and
+         * {@link #performTimeOuts()}
+         */
+        return updateCnt;
       }
 
       queries.clear();
@@ -2254,7 +2289,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             " since a concurrent committed transaction [" + JavaUtils.txnIdToString(rs.getLong(4)) + "," + rs.getLong(5) +
             "] has already updated resouce '" + resourceName + "'";
           LOG.info(msg);
-          if(abortTxns(dbConn, Collections.singletonList(writeSet.get(0).txnId)) != 1) {
+          if(abortTxns(dbConn, Collections.singletonList(writeSet.get(0).txnId), true) != 1) {
             throw new IllegalStateException(msg + " FAILED!");
           }
           dbConn.commit();
@@ -2774,7 +2809,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         close(rs, stmt, null);
         int numTxnsAborted = 0;
         for(List<Long> batchToAbort : timedOutTxns) {
-          if(abortTxns(dbConn, batchToAbort, now - timeout) == batchToAbort.size()) {
+          if(abortTxns(dbConn, batchToAbort, now - timeout, true) == batchToAbort.size()) {
             dbConn.commit();
             numTxnsAborted += batchToAbort.size();
             //todo: add TXNS.COMMENT filed and set it to 'aborted by system due to timeout'

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index bd274ee..dc807df 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
 import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
@@ -120,6 +121,14 @@ public interface TxnStore {
   public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException;
 
   /**
+   * Abort (rollback) a list of transactions in one request.
+   * @param rqst info on transactions to abort
+   * @throws NoSuchTxnException
+   * @throws MetaException
+   */
+  public void abortTxns(AbortTxnsRequest rqst) throws NoSuchTxnException, MetaException;
+
+  /**
    * Commit a transaction
    * @param rqst info on transaction to commit
    * @throws NoSuchTxnException

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 816d8d4..7818043 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -130,6 +130,7 @@ import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.AbortTxnsDesc;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
@@ -437,7 +438,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         return showTxns(txnsDesc);
       }
 
-       LockTableDesc lockTbl = work.getLockTblDesc();
+      AbortTxnsDesc abortTxnsDesc = work.getAbortTxnsDesc();
+      if (abortTxnsDesc != null) {
+        return abortTxns(db, abortTxnsDesc);
+      }
+
+      LockTableDesc lockTbl = work.getLockTblDesc();
       if (lockTbl != null) {
         return lockTable(lockTbl);
       }
@@ -2701,6 +2707,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
+  private int abortTxns(Hive db, AbortTxnsDesc desc) throws HiveException {
+    db.abortTransactions(desc.getTxnids());
+    return 0;
+  }
+
    /**
    * Lock the table/partition specified
    *

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 2467520..20e1ef6 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
@@ -3272,6 +3272,15 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
   }
 
+  public void abortTransactions(List<Long> txnids) throws HiveException {
+    try {
+      getMSC().abortTxns(txnids);
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
   public void createFunction(Function func) throws HiveException {
     try {
       getMSC().createFunction(func);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 731bdf2..f7cd167 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -91,6 +91,7 @@ import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.authorization.AuthorizationParseUtils;
 import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactory;
 import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactoryImpl;
+import org.apache.hadoop.hive.ql.plan.AbortTxnsDesc;
 import org.apache.hadoop.hive.ql.plan.AddPartitionDesc;
 import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.AlterIndexDesc;
@@ -372,6 +373,9 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowTxns(ast);
       break;
+    case HiveParser.TOK_ABORT_TRANSACTIONS:
+      analyzeAbortTxns(ast);
+      break;
     case HiveParser.TOK_SHOWCONF:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowConf(ast);
@@ -2451,6 +2455,21 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     setFetchTask(createFetchTask(desc.getSchema()));
   }
 
+  /**
+   * Add a task to execute "ABORT TRANSACTIONS"
+   * @param ast The parsed command tree
+   * @throws SemanticException Parsing failed
+   */
+  private void analyzeAbortTxns(ASTNode ast) throws SemanticException {
+    List<Long> txnids = new ArrayList<Long>();
+    int numChildren = ast.getChildCount();
+    for (int i = 0; i < numChildren; i++) {
+      txnids.add(Long.parseLong(ast.getChild(i).getText()));
+    }
+    AbortTxnsDesc desc = new AbortTxnsDesc(txnids);
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+  }
+
    /**
    * Add the task according to the parsed command tree. This is used for the CLI
    * command "UNLOCK TABLE ..;".

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index c080124..428e089 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -317,6 +317,7 @@ KW_ISOLATION: 'ISOLATION';
 KW_LEVEL: 'LEVEL';
 KW_SNAPSHOT: 'SNAPSHOT';
 KW_AUTOCOMMIT: 'AUTOCOMMIT';
+KW_ABORT: 'ABORT';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index c62edbf..6c3d42a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -365,6 +365,7 @@ TOK_TXN_READ_WRITE;
 TOK_COMMIT;
 TOK_ROLLBACK;
 TOK_SET_AUTOCOMMIT;
+TOK_ABORT_TRANSACTIONS;
 }
 
 
@@ -509,6 +510,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
     xlateMap.put("KW_UPDATE", "UPDATE");
     xlateMap.put("KW_VALUES", "VALUES");
     xlateMap.put("KW_PURGE", "PURGE");
+    xlateMap.put("KW_ABORT", "ABORT");
+    xlateMap.put("KW_TRANSACTIONS", "TRANSACTIONS");
 
 
     // Operators
@@ -781,6 +784,7 @@ ddlStatement
     | revokeRole
     | setRole
     | showCurrentRole
+    | abortTransactionStatement
     ;
 
 ifExists
@@ -2467,3 +2471,10 @@ setAutoCommitStatement
 /*
 END user defined transaction boundaries
 */
+
+abortTransactionStatement
+@init { pushMsg("abort transactions statement", state); }
+@after { popMsg(state); }
+  :
+  KW_ABORT KW_TRANSACTIONS ( Number )+ -> ^(TOK_ABORT_TRANSACTIONS ( Number )+)
+  ;

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 64af7d1..28bf46b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -623,7 +623,7 @@ principalIdentifier
 //http://www.postgresql.org/docs/9.5/static/sql-keywords-appendix.html
 nonReserved
     :
-    KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS
+    KW_ABORT | KW_ADD | KW_ADMIN | KW_AFTER | KW_ANALYZE | KW_ARCHIVE | KW_ASC | KW_BEFORE | KW_BUCKET | KW_BUCKETS
     | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS
     | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY
     | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY 

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 4a3802d..2a1c377 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -112,6 +112,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERTABLE_PARTCOLTYPE, HiveOperation.ALTERTABLE_PARTCOLTYPE);
     commandType.put(HiveParser.TOK_SHOW_COMPACTIONS, HiveOperation.SHOW_COMPACTIONS);
     commandType.put(HiveParser.TOK_SHOW_TRANSACTIONS, HiveOperation.SHOW_TRANSACTIONS);
+    commandType.put(HiveParser.TOK_ABORT_TRANSACTIONS, HiveOperation.ABORT_TRANSACTIONS);
     commandType.put(HiveParser.TOK_START_TRANSACTION, HiveOperation.START_TRANSACTION);
     commandType.put(HiveParser.TOK_COMMIT, HiveOperation.COMMIT);
     commandType.put(HiveParser.TOK_ROLLBACK, HiveOperation.ROLLBACK);
@@ -239,6 +240,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_SHOWDBLOCKS:
       case HiveParser.TOK_SHOW_COMPACTIONS:
       case HiveParser.TOK_SHOW_TRANSACTIONS:
+      case HiveParser.TOK_ABORT_TRANSACTIONS:
       case HiveParser.TOK_SHOWCONF:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/plan/AbortTxnsDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AbortTxnsDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AbortTxnsDesc.java
new file mode 100644
index 0000000..d82b134
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AbortTxnsDesc.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.plan;
+import java.io.Serializable;
+import java.util.List;
+/**
+ * Descriptor for aborting transactions.
+ */
+public class AbortTxnsDesc extends DDLDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private final List<Long> txnids;
+  /**
+   *  No arg constructor for serialization.
+   */
+  public AbortTxnsDesc(List<Long> txnids) {
+    this.txnids = txnids;
+  }
+  public List<Long> getTxnids() {
+    return txnids;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index 8dbb3c1..80dc940 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -55,6 +55,7 @@ public class DDLWork implements Serializable {
   private ShowLocksDesc showLocksDesc;
   private ShowCompactionsDesc showCompactionsDesc;
   private ShowTxnsDesc showTxnsDesc;
+  private AbortTxnsDesc abortTxnsDesc;
   private DescFunctionDesc descFunctionDesc;
   private ShowPartitionsDesc showPartsDesc;
   private ShowCreateTableDesc showCreateTblDesc;
@@ -346,6 +347,12 @@ public class DDLWork implements Serializable {
     this.showTxnsDesc = showTxnsDesc;
   }
 
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+                 AbortTxnsDesc abortTxnsDesc) {
+    this(inputs, outputs);
+    this.abortTxnsDesc = abortTxnsDesc;
+  }
+
    /**
    * @param descFuncDesc
    */
@@ -744,6 +751,11 @@ public class DDLWork implements Serializable {
     return showTxnsDesc;
   }
 
+  @Explain(displayName = "Abort Transactions Operator", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public AbortTxnsDesc getAbortTxnsDesc() {
+    return abortTxnsDesc;
+  }
+
   /**
    * @return the lockTblDesc
    */
@@ -792,6 +804,10 @@ public class DDLWork implements Serializable {
     this.showTxnsDesc = showTxnsDesc;
   }
 
+  public void setAbortTxnsDesc(AbortTxnsDesc abortTxnsDesc) {
+    this.abortTxnsDesc = abortTxnsDesc;
+  }
+
   /**
    * @param lockTblDesc
    *          the lockTblDesc to set

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index fc6be2b..fd279de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -122,8 +122,8 @@ public enum HiveOperation {
   START_TRANSACTION("START TRANSACTION", null, null, false, false),
   COMMIT("COMMIT", null, null, true, true),
   ROLLBACK("ROLLBACK", null, null, true, true),
-  SET_AUTOCOMMIT("SET AUTOCOMMIT", null, null, true, false);
-  ;
+  SET_AUTOCOMMIT("SET AUTOCOMMIT", null, null, true, false),
+  ABORT_TRANSACTIONS("ABORT TRANSACTIONS", null, null, false, false);
 
   private String operationName;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index 71be469..6b3aa76 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -118,6 +118,7 @@ public enum HiveOperationType {
   ALTERTABLE_COMPACT,
   SHOW_COMPACTIONS,
   SHOW_TRANSACTIONS,
+  ABORT_TRANSACTIONS,
   // ==== Hive command operation types starts here ==== //
   SET,
   RESET,

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index a2aeafc..b8ee72a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -418,7 +418,8 @@ public class Operation2Privilege {
       (null, null));
     op2Priv.put(HiveOperationType.ALTERTABLE_EXCHANGEPARTITION,
       PrivRequirement.newIOPrivRequirement(null, null));
-
+    op2Priv.put(HiveOperationType.ABORT_TRANSACTIONS, PrivRequirement.newIOPrivRequirement
+      (null, null));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/test/queries/clientpositive/dbtxnmgr_abort.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dbtxnmgr_abort.q b/ql/src/test/queries/clientpositive/dbtxnmgr_abort.q
new file mode 100644
index 0000000..d923995
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/dbtxnmgr_abort.q
@@ -0,0 +1,6 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+abort transactions 1234 5678;
+
+abort transactions 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 27
 2 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521
  522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 
 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010;

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/ql/src/test/results/clientpositive/dbtxnmgr_abort.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dbtxnmgr_abort.q.out b/ql/src/test/results/clientpositive/dbtxnmgr_abort.q.out
new file mode 100644
index 0000000..b1c808a
--- /dev/null
+++ b/ql/src/test/results/clientpositive/dbtxnmgr_abort.q.out
@@ -0,0 +1,8 @@
+PREHOOK: query: abort transactions 1234 5678
+PREHOOK: type: ABORT TRANSACTIONS
+POSTHOOK: query: abort transactions 1234 5678
+POSTHOOK: type: ABORT TRANSACTIONS
+PREHOOK: query: abort transactions 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 26
 8 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517
  518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 
 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010
+PREHOOK: type: ABORT TRANSACTIONS
+POSTHOOK: query: abort transactions 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 2
 68 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 51
 7 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766
  767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010
+POSTHOOK: type: ABORT TRANSACTIONS


[4/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 bc417ef..e836154 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
@@ -266,6 +266,8 @@ public class ThriftHiveMetastore {
 
     public void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException;
 
+    public void abort_txns(AbortTxnsRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException;
+
     public void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
 
     public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
@@ -520,6 +522,8 @@ public class ThriftHiveMetastore {
 
     public void abort_txn(AbortTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void abort_txns(AbortTxnsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -3937,6 +3941,29 @@ public class ThriftHiveMetastore {
       return;
     }
 
+    public void abort_txns(AbortTxnsRequest rqst) throws NoSuchTxnException, org.apache.thrift.TException
+    {
+      send_abort_txns(rqst);
+      recv_abort_txns();
+    }
+
+    public void send_abort_txns(AbortTxnsRequest rqst) throws org.apache.thrift.TException
+    {
+      abort_txns_args args = new abort_txns_args();
+      args.setRqst(rqst);
+      sendBase("abort_txns", args);
+    }
+
+    public void recv_abort_txns() throws NoSuchTxnException, org.apache.thrift.TException
+    {
+      abort_txns_result result = new abort_txns_result();
+      receiveBase(result, "abort_txns");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      return;
+    }
+
     public void commit_txn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
     {
       send_commit_txn(rqst);
@@ -8326,6 +8353,38 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void abort_txns(AbortTxnsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      abort_txns_call method_call = new abort_txns_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class abort_txns_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private AbortTxnsRequest rqst;
+      public abort_txns_call(AbortTxnsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("abort_txns", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        abort_txns_args args = new abort_txns_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchTxnException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_abort_txns();
+      }
+    }
+
     public void commit_txn(CommitTxnRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       commit_txn_call method_call = new commit_txn_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
@@ -8864,6 +8923,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_open_txns_info", new get_open_txns_info());
       processMap.put("open_txns", new open_txns());
       processMap.put("abort_txn", new abort_txn());
+      processMap.put("abort_txns", new abort_txns());
       processMap.put("commit_txn", new commit_txn());
       processMap.put("lock", new lock());
       processMap.put("check_lock", new check_lock());
@@ -11831,6 +11891,30 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class abort_txns<I extends Iface> extends org.apache.thrift.ProcessFunction<I, abort_txns_args> {
+      public abort_txns() {
+        super("abort_txns");
+      }
+
+      public abort_txns_args getEmptyArgsInstance() {
+        return new abort_txns_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public abort_txns_result getResult(I iface, abort_txns_args args) throws org.apache.thrift.TException {
+        abort_txns_result result = new abort_txns_result();
+        try {
+          iface.abort_txns(args.rqst);
+        } catch (NoSuchTxnException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     public static class commit_txn<I extends Iface> extends org.apache.thrift.ProcessFunction<I, commit_txn_args> {
       public commit_txn() {
         super("commit_txn");
@@ -12256,6 +12340,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_open_txns_info", new get_open_txns_info());
       processMap.put("open_txns", new open_txns());
       processMap.put("abort_txn", new abort_txn());
+      processMap.put("abort_txns", new abort_txns());
       processMap.put("commit_txn", new commit_txn());
       processMap.put("lock", new lock());
       processMap.put("check_lock", new check_lock());
@@ -19269,6 +19354,62 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class abort_txns<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, abort_txns_args, Void> {
+      public abort_txns() {
+        super("abort_txns");
+      }
+
+      public abort_txns_args getEmptyArgsInstance() {
+        return new abort_txns_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            abort_txns_result result = new abort_txns_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            abort_txns_result result = new abort_txns_result();
+            if (e instanceof NoSuchTxnException) {
+                        result.o1 = (NoSuchTxnException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, abort_txns_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.abort_txns(args.rqst,resultHandler);
+      }
+    }
+
     public static class commit_txn<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, commit_txn_args, Void> {
       public commit_txn() {
         super("commit_txn");
@@ -25398,13 +25539,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list532.size);
-                  String _elem533;
-                  for (int _i534 = 0; _i534 < _list532.size; ++_i534)
+                  org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list540.size);
+                  String _elem541;
+                  for (int _i542 = 0; _i542 < _list540.size; ++_i542)
                   {
-                    _elem533 = iprot.readString();
-                    struct.success.add(_elem533);
+                    _elem541 = iprot.readString();
+                    struct.success.add(_elem541);
                   }
                   iprot.readListEnd();
                 }
@@ -25439,9 +25580,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter535 : struct.success)
+            for (String _iter543 : struct.success)
             {
-              oprot.writeString(_iter535);
+              oprot.writeString(_iter543);
             }
             oprot.writeListEnd();
           }
@@ -25480,9 +25621,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter536 : struct.success)
+            for (String _iter544 : struct.success)
             {
-              oprot.writeString(_iter536);
+              oprot.writeString(_iter544);
             }
           }
         }
@@ -25497,13 +25638,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list537.size);
-            String _elem538;
-            for (int _i539 = 0; _i539 < _list537.size; ++_i539)
+            org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list545.size);
+            String _elem546;
+            for (int _i547 = 0; _i547 < _list545.size; ++_i547)
             {
-              _elem538 = iprot.readString();
-              struct.success.add(_elem538);
+              _elem546 = iprot.readString();
+              struct.success.add(_elem546);
             }
           }
           struct.setSuccessIsSet(true);
@@ -26157,13 +26298,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list540.size);
-                  String _elem541;
-                  for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+                  org.apache.thrift.protocol.TList _list548 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list548.size);
+                  String _elem549;
+                  for (int _i550 = 0; _i550 < _list548.size; ++_i550)
                   {
-                    _elem541 = iprot.readString();
-                    struct.success.add(_elem541);
+                    _elem549 = iprot.readString();
+                    struct.success.add(_elem549);
                   }
                   iprot.readListEnd();
                 }
@@ -26198,9 +26339,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter543 : struct.success)
+            for (String _iter551 : struct.success)
             {
-              oprot.writeString(_iter543);
+              oprot.writeString(_iter551);
             }
             oprot.writeListEnd();
           }
@@ -26239,9 +26380,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter544 : struct.success)
+            for (String _iter552 : struct.success)
             {
-              oprot.writeString(_iter544);
+              oprot.writeString(_iter552);
             }
           }
         }
@@ -26256,13 +26397,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list545 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list545.size);
-            String _elem546;
-            for (int _i547 = 0; _i547 < _list545.size; ++_i547)
+            org.apache.thrift.protocol.TList _list553 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list553.size);
+            String _elem554;
+            for (int _i555 = 0; _i555 < _list553.size; ++_i555)
             {
-              _elem546 = iprot.readString();
-              struct.success.add(_elem546);
+              _elem554 = iprot.readString();
+              struct.success.add(_elem554);
             }
           }
           struct.setSuccessIsSet(true);
@@ -30869,16 +31010,16 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map548 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map548.size);
-                  String _key549;
-                  Type _val550;
-                  for (int _i551 = 0; _i551 < _map548.size; ++_i551)
+                  org.apache.thrift.protocol.TMap _map556 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map556.size);
+                  String _key557;
+                  Type _val558;
+                  for (int _i559 = 0; _i559 < _map556.size; ++_i559)
                   {
-                    _key549 = iprot.readString();
-                    _val550 = new Type();
-                    _val550.read(iprot);
-                    struct.success.put(_key549, _val550);
+                    _key557 = iprot.readString();
+                    _val558 = new Type();
+                    _val558.read(iprot);
+                    struct.success.put(_key557, _val558);
                   }
                   iprot.readMapEnd();
                 }
@@ -30913,10 +31054,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter552 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter560 : struct.success.entrySet())
             {
-              oprot.writeString(_iter552.getKey());
-              _iter552.getValue().write(oprot);
+              oprot.writeString(_iter560.getKey());
+              _iter560.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -30955,10 +31096,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter553 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter561 : struct.success.entrySet())
             {
-              oprot.writeString(_iter553.getKey());
-              _iter553.getValue().write(oprot);
+              oprot.writeString(_iter561.getKey());
+              _iter561.getValue().write(oprot);
             }
           }
         }
@@ -30973,16 +31114,16 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map554 = 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*_map554.size);
-            String _key555;
-            Type _val556;
-            for (int _i557 = 0; _i557 < _map554.size; ++_i557)
+            org.apache.thrift.protocol.TMap _map562 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map562.size);
+            String _key563;
+            Type _val564;
+            for (int _i565 = 0; _i565 < _map562.size; ++_i565)
             {
-              _key555 = iprot.readString();
-              _val556 = new Type();
-              _val556.read(iprot);
-              struct.success.put(_key555, _val556);
+              _key563 = iprot.readString();
+              _val564 = new Type();
+              _val564.read(iprot);
+              struct.success.put(_key563, _val564);
             }
           }
           struct.setSuccessIsSet(true);
@@ -32017,14 +32158,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list558 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list558.size);
-                  FieldSchema _elem559;
-                  for (int _i560 = 0; _i560 < _list558.size; ++_i560)
+                  org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list566.size);
+                  FieldSchema _elem567;
+                  for (int _i568 = 0; _i568 < _list566.size; ++_i568)
                   {
-                    _elem559 = new FieldSchema();
-                    _elem559.read(iprot);
-                    struct.success.add(_elem559);
+                    _elem567 = new FieldSchema();
+                    _elem567.read(iprot);
+                    struct.success.add(_elem567);
                   }
                   iprot.readListEnd();
                 }
@@ -32077,9 +32218,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter561 : struct.success)
+            for (FieldSchema _iter569 : struct.success)
             {
-              _iter561.write(oprot);
+              _iter569.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -32134,9 +32275,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter562 : struct.success)
+            for (FieldSchema _iter570 : struct.success)
             {
-              _iter562.write(oprot);
+              _iter570.write(oprot);
             }
           }
         }
@@ -32157,14 +32298,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list563 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list563.size);
-            FieldSchema _elem564;
-            for (int _i565 = 0; _i565 < _list563.size; ++_i565)
+            org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list571.size);
+            FieldSchema _elem572;
+            for (int _i573 = 0; _i573 < _list571.size; ++_i573)
             {
-              _elem564 = new FieldSchema();
-              _elem564.read(iprot);
-              struct.success.add(_elem564);
+              _elem572 = new FieldSchema();
+              _elem572.read(iprot);
+              struct.success.add(_elem572);
             }
           }
           struct.setSuccessIsSet(true);
@@ -33318,14 +33459,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list566 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list566.size);
-                  FieldSchema _elem567;
-                  for (int _i568 = 0; _i568 < _list566.size; ++_i568)
+                  org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list574.size);
+                  FieldSchema _elem575;
+                  for (int _i576 = 0; _i576 < _list574.size; ++_i576)
                   {
-                    _elem567 = new FieldSchema();
-                    _elem567.read(iprot);
-                    struct.success.add(_elem567);
+                    _elem575 = new FieldSchema();
+                    _elem575.read(iprot);
+                    struct.success.add(_elem575);
                   }
                   iprot.readListEnd();
                 }
@@ -33378,9 +33519,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter569 : struct.success)
+            for (FieldSchema _iter577 : struct.success)
             {
-              _iter569.write(oprot);
+              _iter577.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -33435,9 +33576,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter570 : struct.success)
+            for (FieldSchema _iter578 : struct.success)
             {
-              _iter570.write(oprot);
+              _iter578.write(oprot);
             }
           }
         }
@@ -33458,14 +33599,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list571 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list571.size);
-            FieldSchema _elem572;
-            for (int _i573 = 0; _i573 < _list571.size; ++_i573)
+            org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list579.size);
+            FieldSchema _elem580;
+            for (int _i581 = 0; _i581 < _list579.size; ++_i581)
             {
-              _elem572 = new FieldSchema();
-              _elem572.read(iprot);
-              struct.success.add(_elem572);
+              _elem580 = new FieldSchema();
+              _elem580.read(iprot);
+              struct.success.add(_elem580);
             }
           }
           struct.setSuccessIsSet(true);
@@ -34510,14 +34651,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list574 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list574.size);
-                  FieldSchema _elem575;
-                  for (int _i576 = 0; _i576 < _list574.size; ++_i576)
+                  org.apache.thrift.protocol.TList _list582 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list582.size);
+                  FieldSchema _elem583;
+                  for (int _i584 = 0; _i584 < _list582.size; ++_i584)
                   {
-                    _elem575 = new FieldSchema();
-                    _elem575.read(iprot);
-                    struct.success.add(_elem575);
+                    _elem583 = new FieldSchema();
+                    _elem583.read(iprot);
+                    struct.success.add(_elem583);
                   }
                   iprot.readListEnd();
                 }
@@ -34570,9 +34711,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter577 : struct.success)
+            for (FieldSchema _iter585 : struct.success)
             {
-              _iter577.write(oprot);
+              _iter585.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -34627,9 +34768,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter578 : struct.success)
+            for (FieldSchema _iter586 : struct.success)
             {
-              _iter578.write(oprot);
+              _iter586.write(oprot);
             }
           }
         }
@@ -34650,14 +34791,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list579 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list579.size);
-            FieldSchema _elem580;
-            for (int _i581 = 0; _i581 < _list579.size; ++_i581)
+            org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list587.size);
+            FieldSchema _elem588;
+            for (int _i589 = 0; _i589 < _list587.size; ++_i589)
             {
-              _elem580 = new FieldSchema();
-              _elem580.read(iprot);
-              struct.success.add(_elem580);
+              _elem588 = new FieldSchema();
+              _elem588.read(iprot);
+              struct.success.add(_elem588);
             }
           }
           struct.setSuccessIsSet(true);
@@ -35811,14 +35952,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list582 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list582.size);
-                  FieldSchema _elem583;
-                  for (int _i584 = 0; _i584 < _list582.size; ++_i584)
+                  org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list590.size);
+                  FieldSchema _elem591;
+                  for (int _i592 = 0; _i592 < _list590.size; ++_i592)
                   {
-                    _elem583 = new FieldSchema();
-                    _elem583.read(iprot);
-                    struct.success.add(_elem583);
+                    _elem591 = new FieldSchema();
+                    _elem591.read(iprot);
+                    struct.success.add(_elem591);
                   }
                   iprot.readListEnd();
                 }
@@ -35871,9 +36012,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter585 : struct.success)
+            for (FieldSchema _iter593 : struct.success)
             {
-              _iter585.write(oprot);
+              _iter593.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -35928,9 +36069,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter586 : struct.success)
+            for (FieldSchema _iter594 : struct.success)
             {
-              _iter586.write(oprot);
+              _iter594.write(oprot);
             }
           }
         }
@@ -35951,14 +36092,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list587 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list587.size);
-            FieldSchema _elem588;
-            for (int _i589 = 0; _i589 < _list587.size; ++_i589)
+            org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list595.size);
+            FieldSchema _elem596;
+            for (int _i597 = 0; _i597 < _list595.size; ++_i597)
             {
-              _elem588 = new FieldSchema();
-              _elem588.read(iprot);
-              struct.success.add(_elem588);
+              _elem596 = new FieldSchema();
+              _elem596.read(iprot);
+              struct.success.add(_elem596);
             }
           }
           struct.setSuccessIsSet(true);
@@ -41198,13 +41339,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list590.size);
-                  String _elem591;
-                  for (int _i592 = 0; _i592 < _list590.size; ++_i592)
+                  org.apache.thrift.protocol.TList _list598 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list598.size);
+                  String _elem599;
+                  for (int _i600 = 0; _i600 < _list598.size; ++_i600)
                   {
-                    _elem591 = iprot.readString();
-                    struct.success.add(_elem591);
+                    _elem599 = iprot.readString();
+                    struct.success.add(_elem599);
                   }
                   iprot.readListEnd();
                 }
@@ -41239,9 +41380,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter593 : struct.success)
+            for (String _iter601 : struct.success)
             {
-              oprot.writeString(_iter593);
+              oprot.writeString(_iter601);
             }
             oprot.writeListEnd();
           }
@@ -41280,9 +41421,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter594 : struct.success)
+            for (String _iter602 : struct.success)
             {
-              oprot.writeString(_iter594);
+              oprot.writeString(_iter602);
             }
           }
         }
@@ -41297,13 +41438,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list595.size);
-            String _elem596;
-            for (int _i597 = 0; _i597 < _list595.size; ++_i597)
+            org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list603.size);
+            String _elem604;
+            for (int _i605 = 0; _i605 < _list603.size; ++_i605)
             {
-              _elem596 = iprot.readString();
-              struct.success.add(_elem596);
+              _elem604 = iprot.readString();
+              struct.success.add(_elem604);
             }
           }
           struct.setSuccessIsSet(true);
@@ -42069,13 +42210,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list598 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list598.size);
-                  String _elem599;
-                  for (int _i600 = 0; _i600 < _list598.size; ++_i600)
+                  org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list606.size);
+                  String _elem607;
+                  for (int _i608 = 0; _i608 < _list606.size; ++_i608)
                   {
-                    _elem599 = iprot.readString();
-                    struct.success.add(_elem599);
+                    _elem607 = iprot.readString();
+                    struct.success.add(_elem607);
                   }
                   iprot.readListEnd();
                 }
@@ -42110,9 +42251,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter601 : struct.success)
+            for (String _iter609 : struct.success)
             {
-              oprot.writeString(_iter601);
+              oprot.writeString(_iter609);
             }
             oprot.writeListEnd();
           }
@@ -42151,9 +42292,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter602 : struct.success)
+            for (String _iter610 : struct.success)
             {
-              oprot.writeString(_iter602);
+              oprot.writeString(_iter610);
             }
           }
         }
@@ -42168,13 +42309,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list603 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list603.size);
-            String _elem604;
-            for (int _i605 = 0; _i605 < _list603.size; ++_i605)
+            org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list611.size);
+            String _elem612;
+            for (int _i613 = 0; _i613 < _list611.size; ++_i613)
             {
-              _elem604 = iprot.readString();
-              struct.success.add(_elem604);
+              _elem612 = iprot.readString();
+              struct.success.add(_elem612);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43627,13 +43768,13 @@ public class ThriftHiveMetastore {
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list606 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list606.size);
-                  String _elem607;
-                  for (int _i608 = 0; _i608 < _list606.size; ++_i608)
+                  org.apache.thrift.protocol.TList _list614 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list614.size);
+                  String _elem615;
+                  for (int _i616 = 0; _i616 < _list614.size; ++_i616)
                   {
-                    _elem607 = iprot.readString();
-                    struct.tbl_names.add(_elem607);
+                    _elem615 = iprot.readString();
+                    struct.tbl_names.add(_elem615);
                   }
                   iprot.readListEnd();
                 }
@@ -43664,9 +43805,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter609 : struct.tbl_names)
+            for (String _iter617 : struct.tbl_names)
             {
-              oprot.writeString(_iter609);
+              oprot.writeString(_iter617);
             }
             oprot.writeListEnd();
           }
@@ -43703,9 +43844,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter610 : struct.tbl_names)
+            for (String _iter618 : struct.tbl_names)
             {
-              oprot.writeString(_iter610);
+              oprot.writeString(_iter618);
             }
           }
         }
@@ -43721,13 +43862,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list611 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list611.size);
-            String _elem612;
-            for (int _i613 = 0; _i613 < _list611.size; ++_i613)
+            org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list619.size);
+            String _elem620;
+            for (int _i621 = 0; _i621 < _list619.size; ++_i621)
             {
-              _elem612 = iprot.readString();
-              struct.tbl_names.add(_elem612);
+              _elem620 = iprot.readString();
+              struct.tbl_names.add(_elem620);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -44295,14 +44436,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list614 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list614.size);
-                  Table _elem615;
-                  for (int _i616 = 0; _i616 < _list614.size; ++_i616)
+                  org.apache.thrift.protocol.TList _list622 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list622.size);
+                  Table _elem623;
+                  for (int _i624 = 0; _i624 < _list622.size; ++_i624)
                   {
-                    _elem615 = new Table();
-                    _elem615.read(iprot);
-                    struct.success.add(_elem615);
+                    _elem623 = new Table();
+                    _elem623.read(iprot);
+                    struct.success.add(_elem623);
                   }
                   iprot.readListEnd();
                 }
@@ -44355,9 +44496,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter617 : struct.success)
+            for (Table _iter625 : struct.success)
             {
-              _iter617.write(oprot);
+              _iter625.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -44412,9 +44553,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter618 : struct.success)
+            for (Table _iter626 : struct.success)
             {
-              _iter618.write(oprot);
+              _iter626.write(oprot);
             }
           }
         }
@@ -44435,14 +44576,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list619 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list619.size);
-            Table _elem620;
-            for (int _i621 = 0; _i621 < _list619.size; ++_i621)
+            org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list627.size);
+            Table _elem628;
+            for (int _i629 = 0; _i629 < _list627.size; ++_i629)
             {
-              _elem620 = new Table();
-              _elem620.read(iprot);
-              struct.success.add(_elem620);
+              _elem628 = new Table();
+              _elem628.read(iprot);
+              struct.success.add(_elem628);
             }
           }
           struct.setSuccessIsSet(true);
@@ -45588,13 +45729,13 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list622 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list622.size);
-                  String _elem623;
-                  for (int _i624 = 0; _i624 < _list622.size; ++_i624)
+                  org.apache.thrift.protocol.TList _list630 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list630.size);
+                  String _elem631;
+                  for (int _i632 = 0; _i632 < _list630.size; ++_i632)
                   {
-                    _elem623 = iprot.readString();
-                    struct.success.add(_elem623);
+                    _elem631 = iprot.readString();
+                    struct.success.add(_elem631);
                   }
                   iprot.readListEnd();
                 }
@@ -45647,9 +45788,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter625 : struct.success)
+            for (String _iter633 : struct.success)
             {
-              oprot.writeString(_iter625);
+              oprot.writeString(_iter633);
             }
             oprot.writeListEnd();
           }
@@ -45704,9 +45845,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter626 : struct.success)
+            for (String _iter634 : struct.success)
             {
-              oprot.writeString(_iter626);
+              oprot.writeString(_iter634);
             }
           }
         }
@@ -45727,13 +45868,13 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list627 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list627.size);
-            String _elem628;
-            for (int _i629 = 0; _i629 < _list627.size; ++_i629)
+            org.apache.thrift.protocol.TList _list635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list635.size);
+            String _elem636;
+            for (int _i637 = 0; _i637 < _list635.size; ++_i637)
             {
-              _elem628 = iprot.readString();
-              struct.success.add(_elem628);
+              _elem636 = iprot.readString();
+              struct.success.add(_elem636);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51592,14 +51733,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list630 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list630.size);
-                  Partition _elem631;
-                  for (int _i632 = 0; _i632 < _list630.size; ++_i632)
+                  org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list638.size);
+                  Partition _elem639;
+                  for (int _i640 = 0; _i640 < _list638.size; ++_i640)
                   {
-                    _elem631 = new Partition();
-                    _elem631.read(iprot);
-                    struct.new_parts.add(_elem631);
+                    _elem639 = new Partition();
+                    _elem639.read(iprot);
+                    struct.new_parts.add(_elem639);
                   }
                   iprot.readListEnd();
                 }
@@ -51625,9 +51766,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter633 : struct.new_parts)
+            for (Partition _iter641 : struct.new_parts)
             {
-              _iter633.write(oprot);
+              _iter641.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51658,9 +51799,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter634 : struct.new_parts)
+            for (Partition _iter642 : struct.new_parts)
             {
-              _iter634.write(oprot);
+              _iter642.write(oprot);
             }
           }
         }
@@ -51672,14 +51813,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list635 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list635.size);
-            Partition _elem636;
-            for (int _i637 = 0; _i637 < _list635.size; ++_i637)
+            org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list643.size);
+            Partition _elem644;
+            for (int _i645 = 0; _i645 < _list643.size; ++_i645)
             {
-              _elem636 = new Partition();
-              _elem636.read(iprot);
-              struct.new_parts.add(_elem636);
+              _elem644 = new Partition();
+              _elem644.read(iprot);
+              struct.new_parts.add(_elem644);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -52680,14 +52821,14 @@ public class ThriftHiveMetastore {
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list638.size);
-                  PartitionSpec _elem639;
-                  for (int _i640 = 0; _i640 < _list638.size; ++_i640)
+                  org.apache.thrift.protocol.TList _list646 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<PartitionSpec>(_list646.size);
+                  PartitionSpec _elem647;
+                  for (int _i648 = 0; _i648 < _list646.size; ++_i648)
                   {
-                    _elem639 = new PartitionSpec();
-                    _elem639.read(iprot);
-                    struct.new_parts.add(_elem639);
+                    _elem647 = new PartitionSpec();
+                    _elem647.read(iprot);
+                    struct.new_parts.add(_elem647);
                   }
                   iprot.readListEnd();
                 }
@@ -52713,9 +52854,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (PartitionSpec _iter641 : struct.new_parts)
+            for (PartitionSpec _iter649 : struct.new_parts)
             {
-              _iter641.write(oprot);
+              _iter649.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52746,9 +52887,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (PartitionSpec _iter642 : struct.new_parts)
+            for (PartitionSpec _iter650 : struct.new_parts)
             {
-              _iter642.write(oprot);
+              _iter650.write(oprot);
             }
           }
         }
@@ -52760,14 +52901,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<PartitionSpec>(_list643.size);
-            PartitionSpec _elem644;
-            for (int _i645 = 0; _i645 < _list643.size; ++_i645)
+            org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<PartitionSpec>(_list651.size);
+            PartitionSpec _elem652;
+            for (int _i653 = 0; _i653 < _list651.size; ++_i653)
             {
-              _elem644 = new PartitionSpec();
-              _elem644.read(iprot);
-              struct.new_parts.add(_elem644);
+              _elem652 = new PartitionSpec();
+              _elem652.read(iprot);
+              struct.new_parts.add(_elem652);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -53943,13 +54084,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list646 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list646.size);
-                  String _elem647;
-                  for (int _i648 = 0; _i648 < _list646.size; ++_i648)
+                  org.apache.thrift.protocol.TList _list654 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list654.size);
+                  String _elem655;
+                  for (int _i656 = 0; _i656 < _list654.size; ++_i656)
                   {
-                    _elem647 = iprot.readString();
-                    struct.part_vals.add(_elem647);
+                    _elem655 = iprot.readString();
+                    struct.part_vals.add(_elem655);
                   }
                   iprot.readListEnd();
                 }
@@ -53985,9 +54126,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter649 : struct.part_vals)
+            for (String _iter657 : struct.part_vals)
             {
-              oprot.writeString(_iter649);
+              oprot.writeString(_iter657);
             }
             oprot.writeListEnd();
           }
@@ -54030,9 +54171,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter650 : struct.part_vals)
+            for (String _iter658 : struct.part_vals)
             {
-              oprot.writeString(_iter650);
+              oprot.writeString(_iter658);
             }
           }
         }
@@ -54052,13 +54193,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list651 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list651.size);
-            String _elem652;
-            for (int _i653 = 0; _i653 < _list651.size; ++_i653)
+            org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list659.size);
+            String _elem660;
+            for (int _i661 = 0; _i661 < _list659.size; ++_i661)
             {
-              _elem652 = iprot.readString();
-              struct.part_vals.add(_elem652);
+              _elem660 = iprot.readString();
+              struct.part_vals.add(_elem660);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -56367,13 +56508,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list654 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list654.size);
-                  String _elem655;
-                  for (int _i656 = 0; _i656 < _list654.size; ++_i656)
+                  org.apache.thrift.protocol.TList _list662 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list662.size);
+                  String _elem663;
+                  for (int _i664 = 0; _i664 < _list662.size; ++_i664)
                   {
-                    _elem655 = iprot.readString();
-                    struct.part_vals.add(_elem655);
+                    _elem663 = iprot.readString();
+                    struct.part_vals.add(_elem663);
                   }
                   iprot.readListEnd();
                 }
@@ -56418,9 +56559,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter657 : struct.part_vals)
+            for (String _iter665 : struct.part_vals)
             {
-              oprot.writeString(_iter657);
+              oprot.writeString(_iter665);
             }
             oprot.writeListEnd();
           }
@@ -56471,9 +56612,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter658 : struct.part_vals)
+            for (String _iter666 : struct.part_vals)
             {
-              oprot.writeString(_iter658);
+              oprot.writeString(_iter666);
             }
           }
         }
@@ -56496,13 +56637,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list659 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list659.size);
-            String _elem660;
-            for (int _i661 = 0; _i661 < _list659.size; ++_i661)
+            org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list667.size);
+            String _elem668;
+            for (int _i669 = 0; _i669 < _list667.size; ++_i669)
             {
-              _elem660 = iprot.readString();
-              struct.part_vals.add(_elem660);
+              _elem668 = iprot.readString();
+              struct.part_vals.add(_elem668);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -60372,13 +60513,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list662 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list662.size);
-                  String _elem663;
-                  for (int _i664 = 0; _i664 < _list662.size; ++_i664)
+                  org.apache.thrift.protocol.TList _list670 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list670.size);
+                  String _elem671;
+                  for (int _i672 = 0; _i672 < _list670.size; ++_i672)
                   {
-                    _elem663 = iprot.readString();
-                    struct.part_vals.add(_elem663);
+                    _elem671 = iprot.readString();
+                    struct.part_vals.add(_elem671);
                   }
                   iprot.readListEnd();
                 }
@@ -60422,9 +60563,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter665 : struct.part_vals)
+            for (String _iter673 : struct.part_vals)
             {
-              oprot.writeString(_iter665);
+              oprot.writeString(_iter673);
             }
             oprot.writeListEnd();
           }
@@ -60473,9 +60614,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter666 : struct.part_vals)
+            for (String _iter674 : struct.part_vals)
             {
-              oprot.writeString(_iter666);
+              oprot.writeString(_iter674);
             }
           }
         }
@@ -60498,13 +60639,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list667 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list667.size);
-            String _elem668;
-            for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+            org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list675.size);
+            String _elem676;
+            for (int _i677 = 0; _i677 < _list675.size; ++_i677)
             {
-              _elem668 = iprot.readString();
-              struct.part_vals.add(_elem668);
+              _elem676 = iprot.readString();
+              struct.part_vals.add(_elem676);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -61743,13 +61884,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list670 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list670.size);
-                  String _elem671;
-                  for (int _i672 = 0; _i672 < _list670.size; ++_i672)
+                  org.apache.thrift.protocol.TList _list678 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list678.size);
+                  String _elem679;
+                  for (int _i680 = 0; _i680 < _list678.size; ++_i680)
                   {
-                    _elem671 = iprot.readString();
-                    struct.part_vals.add(_elem671);
+                    _elem679 = iprot.readString();
+                    struct.part_vals.add(_elem679);
                   }
                   iprot.readListEnd();
                 }
@@ -61802,9 +61943,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter673 : struct.part_vals)
+            for (String _iter681 : struct.part_vals)
             {
-              oprot.writeString(_iter673);
+              oprot.writeString(_iter681);
             }
             oprot.writeListEnd();
           }
@@ -61861,9 +62002,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter674 : struct.part_vals)
+            for (String _iter682 : struct.part_vals)
             {
-              oprot.writeString(_iter674);
+              oprot.writeString(_iter682);
             }
           }
         }
@@ -61889,13 +62030,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list675 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list675.size);
-            String _elem676;
-            for (int _i677 = 0; _i677 < _list675.size; ++_i677)
+            org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list683.size);
+            String _elem684;
+            for (int _i685 = 0; _i685 < _list683.size; ++_i685)
             {
-              _elem676 = iprot.readString();
-              struct.part_vals.add(_elem676);
+              _elem684 = iprot.readString();
+              struct.part_vals.add(_elem684);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -66497,13 +66638,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list678 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list678.size);
-                  String _elem679;
-                  for (int _i680 = 0; _i680 < _list678.size; ++_i680)
+                  org.apache.thrift.protocol.TList _list686 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list686.size);
+                  String _elem687;
+                  for (int _i688 = 0; _i688 < _list686.size; ++_i688)
                   {
-                    _elem679 = iprot.readString();
-                    struct.part_vals.add(_elem679);
+                    _elem687 = iprot.readString();
+                    struct.part_vals.add(_elem687);
                   }
                   iprot.readListEnd();
                 }
@@ -66539,9 +66680,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter681 : struct.part_vals)
+            for (String _iter689 : struct.part_vals)
             {
-              oprot.writeString(_iter681);
+              oprot.writeString(_iter689);
             }
             oprot.writeListEnd();
           }
@@ -66584,9 +66725,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter682 : struct.part_vals)
+            for (String _iter690 : struct.part_vals)
             {
-              oprot.writeString(_iter682);
+              oprot.writeString(_iter690);
             }
           }
         }
@@ -66606,13 +66747,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list683 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list683.size);
-            String _elem684;
-            for (int _i685 = 0; _i685 < _list683.size; ++_i685)
+            org.apache.thrift.protocol.TList _list691 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list691.size);
+            String _elem692;
+            for (int _i693 = 0; _i693 < _list691.size; ++_i693)
             {
-              _elem684 = iprot.readString();
-              struct.part_vals.add(_elem684);
+              _elem692 = iprot.readString();
+              struct.part_vals.add(_elem692);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -67830,15 +67971,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map686 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map686.size);
-                  String _key687;
-                  String _val688;
-                  for (int _i689 = 0; _i689 < _map686.size; ++_i689)
+                  org.apache.thrift.protocol.TMap _map694 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map694.size);
+                  String _key695;
+                  String _val696;
+                  for (int _i697 = 0; _i697 < _map694.size; ++_i697)
                   {
-                    _key687 = iprot.readString();
-                    _val688 = iprot.readString();
-                    struct.partitionSpecs.put(_key687, _val688);
+                    _key695 = iprot.readString();
+                    _val696 = iprot.readString();
+                    struct.partitionSpecs.put(_key695, _val696);
                   }
                   iprot.readMapEnd();
                 }
@@ -67896,10 +68037,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter690 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter698 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter690.getKey());
-              oprot.writeString(_iter690.getValue());
+              oprot.writeString(_iter698.getKey());
+              oprot.writeString(_iter698.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -67962,10 +68103,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter691 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter699 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter691.getKey());
-              oprot.writeString(_iter691.getValue());
+              oprot.writeString(_iter699.getKey());
+              oprot.writeString(_iter699.getValue());
             }
           }
         }
@@ -67989,15 +68130,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map692 = 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*_map692.size);
-            String _key693;
-            String _val694;
-            for (int _i695 = 0; _i695 < _map692.size; ++_i695)
+            org.apache.thrift.protocol.TMap _map700 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map700.size);
+            String _key701;
+            String _val702;
+            for (int _i703 = 0; _i703 < _map700.size; ++_i703)
             {
-              _key693 = iprot.readString();
-              _val694 = iprot.readString();
-              struct.partitionSpecs.put(_key693, _val694);
+              _key701 = iprot.readString();
+              _val702 = iprot.readString();
+              struct.partitionSpecs.put(_key701, _val702);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -69443,15 +69584,15 @@ public class ThriftHiveMetastore {
             case 1: // PARTITION_SPECS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map696 = iprot.readMapBegin();
-                  struct.partitionSpecs = new HashMap<String,String>(2*_map696.size);
-                  String _key697;
-                  String _val698;
-                  for (int _i699 = 0; _i699 < _map696.size; ++_i699)
+                  org.apache.thrift.protocol.TMap _map704 = iprot.readMapBegin();
+                  struct.partitionSpecs = new HashMap<String,String>(2*_map704.size);
+                  String _key705;
+                  String _val706;
+                  for (int _i707 = 0; _i707 < _map704.size; ++_i707)
                   {
-                    _key697 = iprot.readString();
-                    _val698 = iprot.readString();
-                    struct.partitionSpecs.put(_key697, _val698);
+                    _key705 = iprot.readString();
+                    _val706 = iprot.readString();
+                    struct.partitionSpecs.put(_key705, _val706);
                   }
                   iprot.readMapEnd();
                 }
@@ -69509,10 +69650,10 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PARTITION_SPECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.partitionSpecs.size()));
-            for (Map.Entry<String, String> _iter700 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter708 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter700.getKey());
-              oprot.writeString(_iter700.getValue());
+              oprot.writeString(_iter708.getKey());
+              oprot.writeString(_iter708.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -69575,10 +69716,10 @@ public class ThriftHiveMetastore {
         if (struct.isSetPartitionSpecs()) {
           {
             oprot.writeI32(struct.partitionSpecs.size());
-            for (Map.Entry<String, String> _iter701 : struct.partitionSpecs.entrySet())
+            for (Map.Entry<String, String> _iter709 : struct.partitionSpecs.entrySet())
             {
-              oprot.writeString(_iter701.getKey());
-              oprot.writeString(_iter701.getValue());
+              oprot.writeString(_iter709.getKey());
+              oprot.writeString(_iter709.getValue());
             }
           }
         }
@@ -69602,15 +69743,15 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map702 = 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*_map702.size);
-            String _key703;
-            String _val704;
-            for (int _i705 = 0; _i705 < _map702.size; ++_i705)
+            org.apache.thrift.protocol.TMap _map710 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partitionSpecs = new HashMap<String,String>(2*_map710.size);
+            String _key711;
+            String _val712;
+            for (int _i713 = 0; _i713 < _map710.size; ++_i713)
             {
-              _key703 = iprot.readString();
-              _val704 = iprot.readString();
-              struct.partitionSpecs.put(_key703, _val704);
+              _key711 = iprot.readString();
+              _val712 = iprot.readString();
+              struct.partitionSpecs.put(_key711, _val712);
             }
           }
           struct.setPartitionSpecsIsSet(true);
@@ -70275,14 +70416,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list706.size);
-                  Partition _elem707;
-                  for (int _i708 = 0; _i708 < _list706.size; ++_i708)
+                  org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list714.size);
+                  Partition _elem715;
+                  for (int _i716 = 0; _i716 < _list714.size; ++_i716)
                   {
-                    _elem707 = new Partition();
-                    _elem707.read(iprot);
-                    struct.success.add(_elem707);
+                    _elem715 = new Partition();
+                    _elem715.read(iprot);
+                    struct.success.add(_elem715);
                   }
                   iprot.readListEnd();
                 }
@@ -70344,9 +70485,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter709 : struct.success)
+            for (Partition _iter717 : struct.success)
             {
-              _iter709.write(oprot);
+              _iter717.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -70409,9 +70550,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter710 : struct.success)
+            for (Partition _iter718 : struct.success)
             {
-              _iter710.write(oprot);
+              _iter718.write(oprot);
             }
           }
         }
@@ -70435,14 +70576,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(5);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list711.size);
-            Partition _elem712;
-            for (int _i713 = 0; _i713 < _list711.size; ++_i713)
+            org.apache.thrift.protocol.TList _list719 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list719.size);
+            Partition _elem720;
+            for (int _i721 = 0; _i721 < _list719.size; ++_i721)
             {
-              _elem712 = new Partition();
-              _elem712.read(iprot);
-              struct.success.add(_elem712);
+              _elem720 = new Partition();
+              _elem720.read(iprot);
+              struct.success.add(_elem720);
             }
           }
           struct.setSuccessIsSet(true);
@@ -71141,13 +71282,13 @@ public class ThriftHiveMetastore {
             case 3: // PART_VALS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list714 = iprot.readListBegin();
-                  struct.part_vals = new ArrayList<String>(_list714.size);
-                  String _elem715;
-                  for (int _i716 = 0; _i716 < _list714.size; ++_i716)
+                  org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
+                  struct.part_vals = new ArrayList<String>(_list722.size);
+                  String _elem723;
+                  for (int _i724 = 0; _i724 < _list722.size; ++_i724)
                   {
-                    _elem715 = iprot.readString();
-                    struct.part_vals.add(_elem715);
+                    _elem723 = iprot.readString();
+                    struct.part_vals.add(_elem723);
                   }
                   iprot.readListEnd();
                 }
@@ -71167,13 +71308,13 @@ public class ThriftHiveMetastore {
             case 5: // GROUP_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list717 = iprot.readListBegin();
-                  struct.group_names = new ArrayList<String>(_list717.size);
-                  String _elem718;
-                  for (int _i719 = 0; _i719 < _list717.size; ++_i719)
+                  org.apache.thrift.protocol.TList _list725 = iprot.readListBegin();
+                  struct.group_names = new ArrayList<String>(_list725.size);
+                  String _elem726;
+                  for (int _i727 = 0; _i727 < _list725.size; ++_i727)
                   {
-                    _elem718 = iprot.readString();
-                    struct.group_names.add(_elem718);
+                    _elem726 = iprot.readString();
+                    struct.group_names.add(_elem726);
                   }
                   iprot.readListEnd();
                 }
@@ -71209,9 +71350,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(PART_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.part_vals.size()));
-            for (String _iter720 : struct.part_vals)
+            for (String _iter728 : struct.part_vals)
             {
-              oprot.writeString(_iter720);
+              oprot.writeString(_iter728);
             }
             oprot.writeListEnd();
           }
@@ -71226,9 +71367,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.group_names.size()));
-            for (String _iter721 : struct.group_names)
+            for (String _iter729 : struct.group_names)
             {
-              oprot.writeString(_iter721);
+              oprot.writeString(_iter729);
             }
             oprot.writeListEnd();
           }
@@ -71277,9 +71418,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetPart_vals()) {
           {
             oprot.writeI32(struct.part_vals.size());
-            for (String _iter722 : struct.part_vals)
+            for (String _iter730 : struct.part_vals)
             {
-              oprot.writeString(_iter722);
+              oprot.writeString(_iter730);
             }
           }
         }
@@ -71289,9 +71430,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetGroup_names()) {
           {
             oprot.writeI32(struct.group_names.size());
-            for (String _iter723 : struct.group_names)
+            for (String _iter731 : struct.group_names)
             {
-              oprot.writeString(_iter723);
+              oprot.writeString(_iter731);
             }
           }
         }
@@ -71311,13 +71452,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list724 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.part_vals = new ArrayList<String>(_list724.size);
-            String _elem725;
-            for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+            org.apache.thrift.protocol.TList _list732 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.part_vals = new ArrayList<String>(_list732.size);
+            String _elem733;
+            for (int _i734 = 0; _i734 < _list732.size; ++_i734)
             {
-              _elem725 = iprot.readString();
-              struct.part_vals.add(_elem725);
+              _elem733 = iprot.readString();
+              struct.part_vals.add(_elem733);
             }
           }
           struct.setPart_valsIsSet(true);
@@ -71328,13 +71469,13 @@ public class ThriftHiveMetastore {
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.group_names = new ArrayList<String>(_list727.size);
-            String _elem728;
-            for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+            org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.group_names = new ArrayList<String>(_list735.size);
+            String _elem736;
+            for (int _i737 = 0; _i737 < _list735.size; ++_i737)
             {
-              _elem728 = iprot.readString();
-              struct.group_names.add(_elem728);
+              _elem736 = iprot.readString();
+              struct.group_names.add(_elem736);
             }
           }
           struct.setGroup_namesIsSet(true);
@@ -74103,14 +74244,14 @@ public class ThriftHiveMetastore {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list730 = iprot.readListBegin();
-                  struct.success = new ArrayList<Partition>(_list730.size);
-                  Partition _elem731;
-                  for (int _i732 = 0; _i732 < _list730.size; ++_i732)
+                  org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
+                  struct.success = new ArrayList<Partition>(_list738.size);
+                  Partition _elem739;
+                  for (int _i740 = 0; _i740 < _list738.size; ++_i740)
                   {
-                    _elem731 = new Partition();
-                    _elem731.read(iprot);
-                    struct.success.add(_elem731);
+                    _elem739 = new Partition();
+                    _elem739.read(iprot);
+                    struct.success.add(_elem739);
                   }
                   iprot.readListEnd();
                 }
@@ -74154,9 +74295,9 @@ public class ThriftHiveMetastore {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Partition _iter733 : struct.success)
+            for (Partition _iter741 : struct.success)
             {
-              _iter733.write(oprot);
+              _iter741.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -74203,9 +74344,9 @@ public class ThriftHiveMetastore {
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Partition _iter734 : struct.success)
+            for (Partition _iter742 : struct.success)
             {
-              _iter734.write(oprot);
+              _iter742.write(oprot);
             }
           }
         }
@@ -74223,14 +74364,14 @@ public class ThriftHiveMetastore {
         BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list735 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Partition>(_list735.size);
-            Partition _elem736;
-            for (int _i737 = 0; _i737 < _list735.size; ++_i737)
+            org.apache.thrift.protocol.TList _list743 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Partition>(_list743.size);
+            Partition _elem744;
+            for (int _i745 = 0; _i745 < _list743.size; ++_i745)
             {
-              _elem736 = new Partition();
-              _elem736.read(iprot);
-              struct.success.add(_elem736);
+ 

<TRUNCATED>

[5/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
new file mode 100644
index 0000000..1bf4655
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AbortTxnsRequest.java
@@ -0,0 +1,438 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class AbortTxnsRequest implements org.apache.thrift.TBase<AbortTxnsRequest, AbortTxnsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AbortTxnsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AbortTxnsRequest");
+
+  private static final org.apache.thrift.protocol.TField TXN_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("txn_ids", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AbortTxnsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AbortTxnsRequestTupleSchemeFactory());
+  }
+
+  private List<Long> txn_ids; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TXN_IDS((short)1, "txn_ids");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TXN_IDS
+          return TXN_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TXN_IDS, new org.apache.thrift.meta_data.FieldMetaData("txn_ids", 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.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AbortTxnsRequest.class, metaDataMap);
+  }
+
+  public AbortTxnsRequest() {
+  }
+
+  public AbortTxnsRequest(
+    List<Long> txn_ids)
+  {
+    this();
+    this.txn_ids = txn_ids;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AbortTxnsRequest(AbortTxnsRequest other) {
+    if (other.isSetTxn_ids()) {
+      List<Long> __this__txn_ids = new ArrayList<Long>(other.txn_ids);
+      this.txn_ids = __this__txn_ids;
+    }
+  }
+
+  public AbortTxnsRequest deepCopy() {
+    return new AbortTxnsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.txn_ids = null;
+  }
+
+  public int getTxn_idsSize() {
+    return (this.txn_ids == null) ? 0 : this.txn_ids.size();
+  }
+
+  public java.util.Iterator<Long> getTxn_idsIterator() {
+    return (this.txn_ids == null) ? null : this.txn_ids.iterator();
+  }
+
+  public void addToTxn_ids(long elem) {
+    if (this.txn_ids == null) {
+      this.txn_ids = new ArrayList<Long>();
+    }
+    this.txn_ids.add(elem);
+  }
+
+  public List<Long> getTxn_ids() {
+    return this.txn_ids;
+  }
+
+  public void setTxn_ids(List<Long> txn_ids) {
+    this.txn_ids = txn_ids;
+  }
+
+  public void unsetTxn_ids() {
+    this.txn_ids = null;
+  }
+
+  /** Returns true if field txn_ids is set (has been assigned a value) and false otherwise */
+  public boolean isSetTxn_ids() {
+    return this.txn_ids != null;
+  }
+
+  public void setTxn_idsIsSet(boolean value) {
+    if (!value) {
+      this.txn_ids = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TXN_IDS:
+      if (value == null) {
+        unsetTxn_ids();
+      } else {
+        setTxn_ids((List<Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TXN_IDS:
+      return getTxn_ids();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TXN_IDS:
+      return isSetTxn_ids();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AbortTxnsRequest)
+      return this.equals((AbortTxnsRequest)that);
+    return false;
+  }
+
+  public boolean equals(AbortTxnsRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_txn_ids = true && this.isSetTxn_ids();
+    boolean that_present_txn_ids = true && that.isSetTxn_ids();
+    if (this_present_txn_ids || that_present_txn_ids) {
+      if (!(this_present_txn_ids && that_present_txn_ids))
+        return false;
+      if (!this.txn_ids.equals(that.txn_ids))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_txn_ids = true && (isSetTxn_ids());
+    list.add(present_txn_ids);
+    if (present_txn_ids)
+      list.add(txn_ids);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AbortTxnsRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTxn_ids()).compareTo(other.isSetTxn_ids());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTxn_ids()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txn_ids, other.txn_ids);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AbortTxnsRequest(");
+    boolean first = true;
+
+    sb.append("txn_ids:");
+    if (this.txn_ids == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.txn_ids);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetTxn_ids()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'txn_ids' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AbortTxnsRequestStandardSchemeFactory implements SchemeFactory {
+    public AbortTxnsRequestStandardScheme getScheme() {
+      return new AbortTxnsRequestStandardScheme();
+    }
+  }
+
+  private static class AbortTxnsRequestStandardScheme extends StandardScheme<AbortTxnsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TXN_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
+                struct.txn_ids = new ArrayList<Long>(_list452.size);
+                long _elem453;
+                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
+                {
+                  _elem453 = iprot.readI64();
+                  struct.txn_ids.add(_elem453);
+                }
+                iprot.readListEnd();
+              }
+              struct.setTxn_idsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.txn_ids != null) {
+        oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txn_ids.size()));
+          for (long _iter455 : struct.txn_ids)
+          {
+            oprot.writeI64(_iter455);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AbortTxnsRequestTupleSchemeFactory implements SchemeFactory {
+    public AbortTxnsRequestTupleScheme getScheme() {
+      return new AbortTxnsRequestTupleScheme();
+    }
+  }
+
+  private static class AbortTxnsRequestTupleScheme extends TupleScheme<AbortTxnsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.txn_ids.size());
+        for (long _iter456 : struct.txn_ids)
+        {
+          oprot.writeI64(_iter456);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AbortTxnsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.txn_ids = new ArrayList<Long>(_list457.size);
+        long _elem458;
+        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
+        {
+          _elem458 = iprot.readI64();
+          struct.txn_ids.add(_elem458);
+        }
+      }
+      struct.setTxn_idsIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index bb6e584..45140bc 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -630,13 +630,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           case 4: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list492 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list492.size);
-                String _elem493;
-                for (int _i494 = 0; _i494 < _list492.size; ++_i494)
+                org.apache.thrift.protocol.TList _list500 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list500.size);
+                String _elem501;
+                for (int _i502 = 0; _i502 < _list500.size; ++_i502)
                 {
-                  _elem493 = iprot.readString();
-                  struct.partitionnames.add(_elem493);
+                  _elem501 = iprot.readString();
+                  struct.partitionnames.add(_elem501);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter495 : struct.partitionnames)
+          for (String _iter503 : struct.partitionnames)
           {
-            oprot.writeString(_iter495);
+            oprot.writeString(_iter503);
           }
           oprot.writeListEnd();
         }
@@ -705,9 +705,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter496 : struct.partitionnames)
+        for (String _iter504 : struct.partitionnames)
         {
-          oprot.writeString(_iter496);
+          oprot.writeString(_iter504);
         }
       }
     }
@@ -722,13 +722,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list497 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list497.size);
-        String _elem498;
-        for (int _i499 = 0; _i499 < _list497.size; ++_i499)
+        org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list505.size);
+        String _elem506;
+        for (int _i507 = 0; _i507 < _list505.size; ++_i507)
         {
-          _elem498 = iprot.readString();
-          struct.partitionnames.add(_elem498);
+          _elem506 = iprot.readString();
+          struct.partitionnames.add(_elem506);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 6b08234..551c46a 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 _list516 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list516.size);
-                String _elem517;
-                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
+                org.apache.thrift.protocol.TList _list524 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list524.size);
+                String _elem525;
+                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
                 {
-                  _elem517 = iprot.readString();
-                  struct.partitionVals.add(_elem517);
+                  _elem525 = iprot.readString();
+                  struct.partitionVals.add(_elem525);
                 }
                 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 _iter519 : struct.partitionVals)
+            for (String _iter527 : struct.partitionVals)
             {
-              oprot.writeString(_iter519);
+              oprot.writeString(_iter527);
             }
             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 _iter520 : struct.partitionVals)
+          for (String _iter528 : struct.partitionVals)
           {
-            oprot.writeString(_iter520);
+            oprot.writeString(_iter528);
           }
         }
       }
@@ -843,13 +843,13 @@ public class FireEventRequest implements org.apache.thrift.TBase<FireEventReques
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list521.size);
-          String _elem522;
-          for (int _i523 = 0; _i523 < _list521.size; ++_i523)
+          org.apache.thrift.protocol.TList _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list529.size);
+          String _elem530;
+          for (int _i531 = 0; _i531 < _list529.size; ++_i531)
           {
-            _elem522 = iprot.readString();
-            struct.partitionVals.add(_elem522);
+            _elem530 = iprot.readString();
+            struct.partitionVals.add(_elem530);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 c97ea46..f23c8e5 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 _list524 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list524.size);
-                Function _elem525;
-                for (int _i526 = 0; _i526 < _list524.size; ++_i526)
+                org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list532.size);
+                Function _elem533;
+                for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                 {
-                  _elem525 = new Function();
-                  _elem525.read(iprot);
-                  struct.functions.add(_elem525);
+                  _elem533 = new Function();
+                  _elem533.read(iprot);
+                  struct.functions.add(_elem533);
                 }
                 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 _iter527 : struct.functions)
+            for (Function _iter535 : struct.functions)
             {
-              _iter527.write(oprot);
+              _iter535.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 _iter528 : struct.functions)
+          for (Function _iter536 : struct.functions)
           {
-            _iter528.write(oprot);
+            _iter536.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 _list529 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list529.size);
-          Function _elem530;
-          for (int _i531 = 0; _i531 < _list529.size; ++_i531)
+          org.apache.thrift.protocol.TList _list537 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list537.size);
+          Function _elem538;
+          for (int _i539 = 0; _i539 < _list537.size; ++_i539)
           {
-            _elem530 = new Function();
-            _elem530.read(iprot);
-            struct.functions.add(_elem530);
+            _elem538 = new Function();
+            _elem538.read(iprot);
+            struct.functions.add(_elem538);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index b00fb9c..963e46d 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set468 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set468.size);
-                long _elem469;
-                for (int _i470 = 0; _i470 < _set468.size; ++_i470)
+                org.apache.thrift.protocol.TSet _set476 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set476.size);
+                long _elem477;
+                for (int _i478 = 0; _i478 < _set476.size; ++_i478)
                 {
-                  _elem469 = iprot.readI64();
-                  struct.aborted.add(_elem469);
+                  _elem477 = iprot.readI64();
+                  struct.aborted.add(_elem477);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set471 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set471.size);
-                long _elem472;
-                for (int _i473 = 0; _i473 < _set471.size; ++_i473)
+                org.apache.thrift.protocol.TSet _set479 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set479.size);
+                long _elem480;
+                for (int _i481 = 0; _i481 < _set479.size; ++_i481)
                 {
-                  _elem472 = iprot.readI64();
-                  struct.nosuch.add(_elem472);
+                  _elem480 = iprot.readI64();
+                  struct.nosuch.add(_elem480);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter474 : struct.aborted)
+          for (long _iter482 : struct.aborted)
           {
-            oprot.writeI64(_iter474);
+            oprot.writeI64(_iter482);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter475 : struct.nosuch)
+          for (long _iter483 : struct.nosuch)
           {
-            oprot.writeI64(_iter475);
+            oprot.writeI64(_iter483);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter476 : struct.aborted)
+        for (long _iter484 : struct.aborted)
         {
-          oprot.writeI64(_iter476);
+          oprot.writeI64(_iter484);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter477 : struct.nosuch)
+        for (long _iter485 : struct.nosuch)
         {
-          oprot.writeI64(_iter477);
+          oprot.writeI64(_iter485);
         }
       }
     }
@@ -560,24 +560,24 @@ public class HeartbeatTxnRangeResponse implements org.apache.thrift.TBase<Heartb
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set478 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set478.size);
-        long _elem479;
-        for (int _i480 = 0; _i480 < _set478.size; ++_i480)
+        org.apache.thrift.protocol.TSet _set486 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set486.size);
+        long _elem487;
+        for (int _i488 = 0; _i488 < _set486.size; ++_i488)
         {
-          _elem479 = iprot.readI64();
-          struct.aborted.add(_elem479);
+          _elem487 = iprot.readI64();
+          struct.aborted.add(_elem487);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set481 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set481.size);
-        long _elem482;
-        for (int _i483 = 0; _i483 < _set481.size; ++_i483)
+        org.apache.thrift.protocol.TSet _set489 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set489.size);
+        long _elem490;
+        for (int _i491 = 0; _i491 < _set489.size; ++_i491)
         {
-          _elem482 = iprot.readI64();
-          struct.nosuch.add(_elem482);
+          _elem490 = iprot.readI64();
+          struct.nosuch.add(_elem490);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 488d3a0..8a799c8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -351,13 +351,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
           case 1: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list508 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list508.size);
-                String _elem509;
-                for (int _i510 = 0; _i510 < _list508.size; ++_i510)
+                org.apache.thrift.protocol.TList _list516 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list516.size);
+                String _elem517;
+                for (int _i518 = 0; _i518 < _list516.size; ++_i518)
                 {
-                  _elem509 = iprot.readString();
-                  struct.filesAdded.add(_elem509);
+                  _elem517 = iprot.readString();
+                  struct.filesAdded.add(_elem517);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter511 : struct.filesAdded)
+          for (String _iter519 : struct.filesAdded)
           {
-            oprot.writeString(_iter511);
+            oprot.writeString(_iter519);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter512 : struct.filesAdded)
+        for (String _iter520 : struct.filesAdded)
         {
-          oprot.writeString(_iter512);
+          oprot.writeString(_iter520);
         }
       }
     }
@@ -421,13 +421,13 @@ public class InsertEventRequestData implements org.apache.thrift.TBase<InsertEve
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list513.size);
-        String _elem514;
-        for (int _i515 = 0; _i515 < _list513.size; ++_i515)
+        org.apache.thrift.protocol.TList _list521 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list521.size);
+        String _elem522;
+        for (int _i523 = 0; _i523 < _list521.size; ++_i523)
         {
-          _elem514 = iprot.readString();
-          struct.filesAdded.add(_elem514);
+          _elem522 = iprot.readString();
+          struct.filesAdded.add(_elem522);
         }
       }
       struct.setFilesAddedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index f39f582..922a6e8 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list452 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list452.size);
-                LockComponent _elem453;
-                for (int _i454 = 0; _i454 < _list452.size; ++_i454)
+                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list460.size);
+                LockComponent _elem461;
+                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
                 {
-                  _elem453 = new LockComponent();
-                  _elem453.read(iprot);
-                  struct.component.add(_elem453);
+                  _elem461 = new LockComponent();
+                  _elem461.read(iprot);
+                  struct.component.add(_elem461);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter455 : struct.component)
+          for (LockComponent _iter463 : struct.component)
           {
-            _iter455.write(oprot);
+            _iter463.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter456 : struct.component)
+        for (LockComponent _iter464 : struct.component)
         {
-          _iter456.write(oprot);
+          _iter464.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ public class LockRequest implements org.apache.thrift.TBase<LockRequest, LockReq
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list457.size);
-        LockComponent _elem458;
-        for (int _i459 = 0; _i459 < _list457.size; ++_i459)
+        org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list465.size);
+        LockComponent _elem466;
+        for (int _i467 = 0; _i467 < _list465.size; ++_i467)
         {
-          _elem458 = new LockComponent();
-          _elem458.read(iprot);
-          struct.component.add(_elem458);
+          _elem466 = new LockComponent();
+          _elem466.read(iprot);
+          struct.component.add(_elem466);
         }
       }
       struct.setComponentIsSet(true);

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index b9b7f3c..4a61355 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list460 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list460.size);
-                ShowLocksResponseElement _elem461;
-                for (int _i462 = 0; _i462 < _list460.size; ++_i462)
+                org.apache.thrift.protocol.TList _list468 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list468.size);
+                ShowLocksResponseElement _elem469;
+                for (int _i470 = 0; _i470 < _list468.size; ++_i470)
                 {
-                  _elem461 = new ShowLocksResponseElement();
-                  _elem461.read(iprot);
-                  struct.locks.add(_elem461);
+                  _elem469 = new ShowLocksResponseElement();
+                  _elem469.read(iprot);
+                  struct.locks.add(_elem469);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter463 : struct.locks)
+          for (ShowLocksResponseElement _iter471 : struct.locks)
           {
-            _iter463.write(oprot);
+            _iter471.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter464 : struct.locks)
+          for (ShowLocksResponseElement _iter472 : struct.locks)
           {
-            _iter464.write(oprot);
+            _iter472.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ public class ShowLocksResponse implements org.apache.thrift.TBase<ShowLocksRespo
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list465.size);
-          ShowLocksResponseElement _elem466;
-          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
+          org.apache.thrift.protocol.TList _list473 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list473.size);
+          ShowLocksResponseElement _elem474;
+          for (int _i475 = 0; _i475 < _list473.size; ++_i475)
           {
-            _elem466 = new ShowLocksResponseElement();
-            _elem466.read(iprot);
-            struct.locks.add(_elem466);
+            _elem474 = new ShowLocksResponseElement();
+            _elem474.read(iprot);
+            struct.locks.add(_elem474);
           }
         }
         struct.setLocksIsSet(true);


[6/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index a7f17cd..64b06c4 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -134,6 +134,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_open_txns_info(GetOpenTxnsInfoResponse& _return) = 0;
   virtual void open_txns(OpenTxnsResponse& _return, const OpenTxnRequest& rqst) = 0;
   virtual void abort_txn(const AbortTxnRequest& rqst) = 0;
+  virtual void abort_txns(const AbortTxnsRequest& rqst) = 0;
   virtual void commit_txn(const CommitTxnRequest& rqst) = 0;
   virtual void lock(LockResponse& _return, const LockRequest& rqst) = 0;
   virtual void check_lock(LockResponse& _return, const CheckLockRequest& rqst) = 0;
@@ -535,6 +536,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void abort_txn(const AbortTxnRequest& /* rqst */) {
     return;
   }
+  void abort_txns(const AbortTxnsRequest& /* rqst */) {
+    return;
+  }
   void commit_txn(const CommitTxnRequest& /* rqst */) {
     return;
   }
@@ -14981,6 +14985,110 @@ class ThriftHiveMetastore_abort_txn_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_abort_txns_args__isset {
+  _ThriftHiveMetastore_abort_txns_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_abort_txns_args__isset;
+
+class ThriftHiveMetastore_abort_txns_args {
+ public:
+
+  ThriftHiveMetastore_abort_txns_args(const ThriftHiveMetastore_abort_txns_args&);
+  ThriftHiveMetastore_abort_txns_args& operator=(const ThriftHiveMetastore_abort_txns_args&);
+  ThriftHiveMetastore_abort_txns_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_abort_txns_args() throw();
+  AbortTxnsRequest rqst;
+
+  _ThriftHiveMetastore_abort_txns_args__isset __isset;
+
+  void __set_rqst(const AbortTxnsRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_abort_txns_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_abort_txns_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_abort_txns_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_abort_txns_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_abort_txns_pargs() throw();
+  const AbortTxnsRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_abort_txns_result__isset {
+  _ThriftHiveMetastore_abort_txns_result__isset() : o1(false) {}
+  bool o1 :1;
+} _ThriftHiveMetastore_abort_txns_result__isset;
+
+class ThriftHiveMetastore_abort_txns_result {
+ public:
+
+  ThriftHiveMetastore_abort_txns_result(const ThriftHiveMetastore_abort_txns_result&);
+  ThriftHiveMetastore_abort_txns_result& operator=(const ThriftHiveMetastore_abort_txns_result&);
+  ThriftHiveMetastore_abort_txns_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_abort_txns_result() throw();
+  NoSuchTxnException o1;
+
+  _ThriftHiveMetastore_abort_txns_result__isset __isset;
+
+  void __set_o1(const NoSuchTxnException& val);
+
+  bool operator == (const ThriftHiveMetastore_abort_txns_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_abort_txns_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_abort_txns_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_abort_txns_presult__isset {
+  _ThriftHiveMetastore_abort_txns_presult__isset() : o1(false) {}
+  bool o1 :1;
+} _ThriftHiveMetastore_abort_txns_presult__isset;
+
+class ThriftHiveMetastore_abort_txns_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_abort_txns_presult() throw();
+  NoSuchTxnException o1;
+
+  _ThriftHiveMetastore_abort_txns_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_commit_txn_args__isset {
   _ThriftHiveMetastore_commit_txn_args__isset() : rqst(false) {}
   bool rqst :1;
@@ -16730,6 +16838,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void abort_txn(const AbortTxnRequest& rqst);
   void send_abort_txn(const AbortTxnRequest& rqst);
   void recv_abort_txn();
+  void abort_txns(const AbortTxnsRequest& rqst);
+  void send_abort_txns(const AbortTxnsRequest& rqst);
+  void recv_abort_txns();
   void commit_txn(const CommitTxnRequest& rqst);
   void send_commit_txn(const CommitTxnRequest& rqst);
   void recv_commit_txn();
@@ -16891,6 +17002,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_open_txns_info(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_open_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_abort_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_abort_txns(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_commit_txn(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_check_lock(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -17020,6 +17132,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_open_txns_info"] = &ThriftHiveMetastoreProcessor::process_get_open_txns_info;
     processMap_["open_txns"] = &ThriftHiveMetastoreProcessor::process_open_txns;
     processMap_["abort_txn"] = &ThriftHiveMetastoreProcessor::process_abort_txn;
+    processMap_["abort_txns"] = &ThriftHiveMetastoreProcessor::process_abort_txns;
     processMap_["commit_txn"] = &ThriftHiveMetastoreProcessor::process_commit_txn;
     processMap_["lock"] = &ThriftHiveMetastoreProcessor::process_lock;
     processMap_["check_lock"] = &ThriftHiveMetastoreProcessor::process_check_lock;
@@ -18141,6 +18254,15 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->abort_txn(rqst);
   }
 
+  void abort_txns(const AbortTxnsRequest& rqst) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->abort_txns(rqst);
+    }
+    ifaces_[i]->abort_txns(rqst);
+  }
+
   void commit_txn(const CommitTxnRequest& rqst) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -18618,6 +18740,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void abort_txn(const AbortTxnRequest& rqst);
   int32_t send_abort_txn(const AbortTxnRequest& rqst);
   void recv_abort_txn(const int32_t seqid);
+  void abort_txns(const AbortTxnsRequest& rqst);
+  int32_t send_abort_txns(const AbortTxnsRequest& rqst);
+  void recv_abort_txns(const int32_t seqid);
   void commit_txn(const CommitTxnRequest& rqst);
   int32_t send_commit_txn(const CommitTxnRequest& rqst);
   void recv_commit_txn(const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 32cf67c..9718079 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -582,6 +582,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("abort_txn\n");
   }
 
+  void abort_txns(const AbortTxnsRequest& rqst) {
+    // Your implementation goes here
+    printf("abort_txns\n");
+  }
+
   void commit_txn(const CommitTxnRequest& rqst) {
     // Your implementation goes here
     printf("commit_txn\n");

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 c2fe9db..7ea4493 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -10781,6 +10781,112 @@ void AbortTxnRequest::printTo(std::ostream& out) const {
 }
 
 
+AbortTxnsRequest::~AbortTxnsRequest() throw() {
+}
+
+
+void AbortTxnsRequest::__set_txn_ids(const std::vector<int64_t> & val) {
+  this->txn_ids = val;
+}
+
+uint32_t AbortTxnsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_txn_ids = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->txn_ids.clear();
+            uint32_t _size496;
+            ::apache::thrift::protocol::TType _etype499;
+            xfer += iprot->readListBegin(_etype499, _size496);
+            this->txn_ids.resize(_size496);
+            uint32_t _i500;
+            for (_i500 = 0; _i500 < _size496; ++_i500)
+            {
+              xfer += iprot->readI64(this->txn_ids[_i500]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_txn_ids = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_txn_ids)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t AbortTxnsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("AbortTxnsRequest");
+
+  xfer += oprot->writeFieldBegin("txn_ids", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txn_ids.size()));
+    std::vector<int64_t> ::const_iterator _iter501;
+    for (_iter501 = this->txn_ids.begin(); _iter501 != this->txn_ids.end(); ++_iter501)
+    {
+      xfer += oprot->writeI64((*_iter501));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(AbortTxnsRequest &a, AbortTxnsRequest &b) {
+  using ::std::swap;
+  swap(a.txn_ids, b.txn_ids);
+}
+
+AbortTxnsRequest::AbortTxnsRequest(const AbortTxnsRequest& other502) {
+  txn_ids = other502.txn_ids;
+}
+AbortTxnsRequest& AbortTxnsRequest::operator=(const AbortTxnsRequest& other503) {
+  txn_ids = other503.txn_ids;
+  return *this;
+}
+void AbortTxnsRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "AbortTxnsRequest(";
+  out << "txn_ids=" << to_string(txn_ids);
+  out << ")";
+}
+
+
 CommitTxnRequest::~CommitTxnRequest() throw() {
 }
 
@@ -10852,11 +10958,11 @@ void swap(CommitTxnRequest &a, CommitTxnRequest &b) {
   swap(a.txnid, b.txnid);
 }
 
-CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other496) {
-  txnid = other496.txnid;
+CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other504) {
+  txnid = other504.txnid;
 }
-CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other497) {
-  txnid = other497.txnid;
+CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other505) {
+  txnid = other505.txnid;
   return *this;
 }
 void CommitTxnRequest::printTo(std::ostream& out) const {
@@ -10919,9 +11025,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast498;
-          xfer += iprot->readI32(ecast498);
-          this->type = (LockType::type)ecast498;
+          int32_t ecast506;
+          xfer += iprot->readI32(ecast506);
+          this->type = (LockType::type)ecast506;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -10929,9 +11035,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast499;
-          xfer += iprot->readI32(ecast499);
-          this->level = (LockLevel::type)ecast499;
+          int32_t ecast507;
+          xfer += iprot->readI32(ecast507);
+          this->level = (LockLevel::type)ecast507;
           isset_level = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11021,21 +11127,21 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other500) {
-  type = other500.type;
-  level = other500.level;
-  dbname = other500.dbname;
-  tablename = other500.tablename;
-  partitionname = other500.partitionname;
-  __isset = other500.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other501) {
-  type = other501.type;
-  level = other501.level;
-  dbname = other501.dbname;
-  tablename = other501.tablename;
-  partitionname = other501.partitionname;
-  __isset = other501.__isset;
+LockComponent::LockComponent(const LockComponent& other508) {
+  type = other508.type;
+  level = other508.level;
+  dbname = other508.dbname;
+  tablename = other508.tablename;
+  partitionname = other508.partitionname;
+  __isset = other508.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other509) {
+  type = other509.type;
+  level = other509.level;
+  dbname = other509.dbname;
+  tablename = other509.tablename;
+  partitionname = other509.partitionname;
+  __isset = other509.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -11104,14 +11210,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size502;
-            ::apache::thrift::protocol::TType _etype505;
-            xfer += iprot->readListBegin(_etype505, _size502);
-            this->component.resize(_size502);
-            uint32_t _i506;
-            for (_i506 = 0; _i506 < _size502; ++_i506)
+            uint32_t _size510;
+            ::apache::thrift::protocol::TType _etype513;
+            xfer += iprot->readListBegin(_etype513, _size510);
+            this->component.resize(_size510);
+            uint32_t _i514;
+            for (_i514 = 0; _i514 < _size510; ++_i514)
             {
-              xfer += this->component[_i506].read(iprot);
+              xfer += this->component[_i514].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11178,10 +11284,10 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
-    std::vector<LockComponent> ::const_iterator _iter507;
-    for (_iter507 = this->component.begin(); _iter507 != this->component.end(); ++_iter507)
+    std::vector<LockComponent> ::const_iterator _iter515;
+    for (_iter515 = this->component.begin(); _iter515 != this->component.end(); ++_iter515)
     {
-      xfer += (*_iter507).write(oprot);
+      xfer += (*_iter515).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11220,21 +11326,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other508) {
-  component = other508.component;
-  txnid = other508.txnid;
-  user = other508.user;
-  hostname = other508.hostname;
-  agentInfo = other508.agentInfo;
-  __isset = other508.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other509) {
-  component = other509.component;
-  txnid = other509.txnid;
-  user = other509.user;
-  hostname = other509.hostname;
-  agentInfo = other509.agentInfo;
-  __isset = other509.__isset;
+LockRequest::LockRequest(const LockRequest& other516) {
+  component = other516.component;
+  txnid = other516.txnid;
+  user = other516.user;
+  hostname = other516.hostname;
+  agentInfo = other516.agentInfo;
+  __isset = other516.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other517) {
+  component = other517.component;
+  txnid = other517.txnid;
+  user = other517.user;
+  hostname = other517.hostname;
+  agentInfo = other517.agentInfo;
+  __isset = other517.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -11294,9 +11400,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast510;
-          xfer += iprot->readI32(ecast510);
-          this->state = (LockState::type)ecast510;
+          int32_t ecast518;
+          xfer += iprot->readI32(ecast518);
+          this->state = (LockState::type)ecast518;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11342,13 +11448,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other511) {
-  lockid = other511.lockid;
-  state = other511.state;
+LockResponse::LockResponse(const LockResponse& other519) {
+  lockid = other519.lockid;
+  state = other519.state;
 }
-LockResponse& LockResponse::operator=(const LockResponse& other512) {
-  lockid = other512.lockid;
-  state = other512.state;
+LockResponse& LockResponse::operator=(const LockResponse& other520) {
+  lockid = other520.lockid;
+  state = other520.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -11470,17 +11576,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other513) {
-  lockid = other513.lockid;
-  txnid = other513.txnid;
-  elapsed_ms = other513.elapsed_ms;
-  __isset = other513.__isset;
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other521) {
+  lockid = other521.lockid;
+  txnid = other521.txnid;
+  elapsed_ms = other521.elapsed_ms;
+  __isset = other521.__isset;
 }
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other514) {
-  lockid = other514.lockid;
-  txnid = other514.txnid;
-  elapsed_ms = other514.elapsed_ms;
-  __isset = other514.__isset;
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other522) {
+  lockid = other522.lockid;
+  txnid = other522.txnid;
+  elapsed_ms = other522.elapsed_ms;
+  __isset = other522.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -11564,11 +11670,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other515) {
-  lockid = other515.lockid;
+UnlockRequest::UnlockRequest(const UnlockRequest& other523) {
+  lockid = other523.lockid;
 }
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other516) {
-  lockid = other516.lockid;
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other524) {
+  lockid = other524.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -11707,19 +11813,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other517) {
-  dbname = other517.dbname;
-  tablename = other517.tablename;
-  partname = other517.partname;
-  isExtended = other517.isExtended;
-  __isset = other517.__isset;
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other525) {
+  dbname = other525.dbname;
+  tablename = other525.tablename;
+  partname = other525.partname;
+  isExtended = other525.isExtended;
+  __isset = other525.__isset;
 }
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other518) {
-  dbname = other518.dbname;
-  tablename = other518.tablename;
-  partname = other518.partname;
-  isExtended = other518.isExtended;
-  __isset = other518.__isset;
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other526) {
+  dbname = other526.dbname;
+  tablename = other526.tablename;
+  partname = other526.partname;
+  isExtended = other526.isExtended;
+  __isset = other526.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -11872,9 +11978,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast519;
-          xfer += iprot->readI32(ecast519);
-          this->state = (LockState::type)ecast519;
+          int32_t ecast527;
+          xfer += iprot->readI32(ecast527);
+          this->state = (LockState::type)ecast527;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11882,9 +11988,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast520;
-          xfer += iprot->readI32(ecast520);
-          this->type = (LockType::type)ecast520;
+          int32_t ecast528;
+          xfer += iprot->readI32(ecast528);
+          this->type = (LockType::type)ecast528;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12100,43 +12206,43 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other521) {
-  lockid = other521.lockid;
-  dbname = other521.dbname;
-  tablename = other521.tablename;
-  partname = other521.partname;
-  state = other521.state;
-  type = other521.type;
-  txnid = other521.txnid;
-  lastheartbeat = other521.lastheartbeat;
-  acquiredat = other521.acquiredat;
-  user = other521.user;
-  hostname = other521.hostname;
-  heartbeatCount = other521.heartbeatCount;
-  agentInfo = other521.agentInfo;
-  blockedByExtId = other521.blockedByExtId;
-  blockedByIntId = other521.blockedByIntId;
-  lockIdInternal = other521.lockIdInternal;
-  __isset = other521.__isset;
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other529) {
+  lockid = other529.lockid;
+  dbname = other529.dbname;
+  tablename = other529.tablename;
+  partname = other529.partname;
+  state = other529.state;
+  type = other529.type;
+  txnid = other529.txnid;
+  lastheartbeat = other529.lastheartbeat;
+  acquiredat = other529.acquiredat;
+  user = other529.user;
+  hostname = other529.hostname;
+  heartbeatCount = other529.heartbeatCount;
+  agentInfo = other529.agentInfo;
+  blockedByExtId = other529.blockedByExtId;
+  blockedByIntId = other529.blockedByIntId;
+  lockIdInternal = other529.lockIdInternal;
+  __isset = other529.__isset;
 }
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other522) {
-  lockid = other522.lockid;
-  dbname = other522.dbname;
-  tablename = other522.tablename;
-  partname = other522.partname;
-  state = other522.state;
-  type = other522.type;
-  txnid = other522.txnid;
-  lastheartbeat = other522.lastheartbeat;
-  acquiredat = other522.acquiredat;
-  user = other522.user;
-  hostname = other522.hostname;
-  heartbeatCount = other522.heartbeatCount;
-  agentInfo = other522.agentInfo;
-  blockedByExtId = other522.blockedByExtId;
-  blockedByIntId = other522.blockedByIntId;
-  lockIdInternal = other522.lockIdInternal;
-  __isset = other522.__isset;
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other530) {
+  lockid = other530.lockid;
+  dbname = other530.dbname;
+  tablename = other530.tablename;
+  partname = other530.partname;
+  state = other530.state;
+  type = other530.type;
+  txnid = other530.txnid;
+  lastheartbeat = other530.lastheartbeat;
+  acquiredat = other530.acquiredat;
+  user = other530.user;
+  hostname = other530.hostname;
+  heartbeatCount = other530.heartbeatCount;
+  agentInfo = other530.agentInfo;
+  blockedByExtId = other530.blockedByExtId;
+  blockedByIntId = other530.blockedByIntId;
+  lockIdInternal = other530.lockIdInternal;
+  __isset = other530.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -12195,14 +12301,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size523;
-            ::apache::thrift::protocol::TType _etype526;
-            xfer += iprot->readListBegin(_etype526, _size523);
-            this->locks.resize(_size523);
-            uint32_t _i527;
-            for (_i527 = 0; _i527 < _size523; ++_i527)
+            uint32_t _size531;
+            ::apache::thrift::protocol::TType _etype534;
+            xfer += iprot->readListBegin(_etype534, _size531);
+            this->locks.resize(_size531);
+            uint32_t _i535;
+            for (_i535 = 0; _i535 < _size531; ++_i535)
             {
-              xfer += this->locks[_i527].read(iprot);
+              xfer += this->locks[_i535].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12231,10 +12337,10 @@ uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
-    std::vector<ShowLocksResponseElement> ::const_iterator _iter528;
-    for (_iter528 = this->locks.begin(); _iter528 != this->locks.end(); ++_iter528)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter536;
+    for (_iter536 = this->locks.begin(); _iter536 != this->locks.end(); ++_iter536)
     {
-      xfer += (*_iter528).write(oprot);
+      xfer += (*_iter536).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12251,13 +12357,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other529) {
-  locks = other529.locks;
-  __isset = other529.__isset;
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other537) {
+  locks = other537.locks;
+  __isset = other537.__isset;
 }
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other530) {
-  locks = other530.locks;
-  __isset = other530.__isset;
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other538) {
+  locks = other538.locks;
+  __isset = other538.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -12358,15 +12464,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other531) {
-  lockid = other531.lockid;
-  txnid = other531.txnid;
-  __isset = other531.__isset;
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other539) {
+  lockid = other539.lockid;
+  txnid = other539.txnid;
+  __isset = other539.__isset;
 }
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other532) {
-  lockid = other532.lockid;
-  txnid = other532.txnid;
-  __isset = other532.__isset;
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other540) {
+  lockid = other540.lockid;
+  txnid = other540.txnid;
+  __isset = other540.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -12469,13 +12575,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other533) {
-  min = other533.min;
-  max = other533.max;
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other541) {
+  min = other541.min;
+  max = other541.max;
 }
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other534) {
-  min = other534.min;
-  max = other534.max;
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other542) {
+  min = other542.min;
+  max = other542.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -12526,15 +12632,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size535;
-            ::apache::thrift::protocol::TType _etype538;
-            xfer += iprot->readSetBegin(_etype538, _size535);
-            uint32_t _i539;
-            for (_i539 = 0; _i539 < _size535; ++_i539)
+            uint32_t _size543;
+            ::apache::thrift::protocol::TType _etype546;
+            xfer += iprot->readSetBegin(_etype546, _size543);
+            uint32_t _i547;
+            for (_i547 = 0; _i547 < _size543; ++_i547)
             {
-              int64_t _elem540;
-              xfer += iprot->readI64(_elem540);
-              this->aborted.insert(_elem540);
+              int64_t _elem548;
+              xfer += iprot->readI64(_elem548);
+              this->aborted.insert(_elem548);
             }
             xfer += iprot->readSetEnd();
           }
@@ -12547,15 +12653,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size541;
-            ::apache::thrift::protocol::TType _etype544;
-            xfer += iprot->readSetBegin(_etype544, _size541);
-            uint32_t _i545;
-            for (_i545 = 0; _i545 < _size541; ++_i545)
+            uint32_t _size549;
+            ::apache::thrift::protocol::TType _etype552;
+            xfer += iprot->readSetBegin(_etype552, _size549);
+            uint32_t _i553;
+            for (_i553 = 0; _i553 < _size549; ++_i553)
             {
-              int64_t _elem546;
-              xfer += iprot->readI64(_elem546);
-              this->nosuch.insert(_elem546);
+              int64_t _elem554;
+              xfer += iprot->readI64(_elem554);
+              this->nosuch.insert(_elem554);
             }
             xfer += iprot->readSetEnd();
           }
@@ -12588,10 +12694,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
-    std::set<int64_t> ::const_iterator _iter547;
-    for (_iter547 = this->aborted.begin(); _iter547 != this->aborted.end(); ++_iter547)
+    std::set<int64_t> ::const_iterator _iter555;
+    for (_iter555 = this->aborted.begin(); _iter555 != this->aborted.end(); ++_iter555)
     {
-      xfer += oprot->writeI64((*_iter547));
+      xfer += oprot->writeI64((*_iter555));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -12600,10 +12706,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
-    std::set<int64_t> ::const_iterator _iter548;
-    for (_iter548 = this->nosuch.begin(); _iter548 != this->nosuch.end(); ++_iter548)
+    std::set<int64_t> ::const_iterator _iter556;
+    for (_iter556 = this->nosuch.begin(); _iter556 != this->nosuch.end(); ++_iter556)
     {
-      xfer += oprot->writeI64((*_iter548));
+      xfer += oprot->writeI64((*_iter556));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -12620,13 +12726,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other549) {
-  aborted = other549.aborted;
-  nosuch = other549.nosuch;
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other557) {
+  aborted = other557.aborted;
+  nosuch = other557.nosuch;
 }
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other550) {
-  aborted = other550.aborted;
-  nosuch = other550.nosuch;
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other558) {
+  aborted = other558.aborted;
+  nosuch = other558.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -12714,9 +12820,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast551;
-          xfer += iprot->readI32(ecast551);
-          this->type = (CompactionType::type)ecast551;
+          int32_t ecast559;
+          xfer += iprot->readI32(ecast559);
+          this->type = (CompactionType::type)ecast559;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12790,21 +12896,21 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other552) {
-  dbname = other552.dbname;
-  tablename = other552.tablename;
-  partitionname = other552.partitionname;
-  type = other552.type;
-  runas = other552.runas;
-  __isset = other552.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other553) {
-  dbname = other553.dbname;
-  tablename = other553.tablename;
-  partitionname = other553.partitionname;
-  type = other553.type;
-  runas = other553.runas;
-  __isset = other553.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other560) {
+  dbname = other560.dbname;
+  tablename = other560.tablename;
+  partitionname = other560.partitionname;
+  type = other560.type;
+  runas = other560.runas;
+  __isset = other560.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other561) {
+  dbname = other561.dbname;
+  tablename = other561.tablename;
+  partitionname = other561.partitionname;
+  type = other561.type;
+  runas = other561.runas;
+  __isset = other561.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -12867,11 +12973,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other554) {
-  (void) other554;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other562) {
+  (void) other562;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other555) {
-  (void) other555;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other563) {
+  (void) other563;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -12992,9 +13098,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast556;
-          xfer += iprot->readI32(ecast556);
-          this->type = (CompactionType::type)ecast556;
+          int32_t ecast564;
+          xfer += iprot->readI32(ecast564);
+          this->type = (CompactionType::type)ecast564;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13167,35 +13273,35 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other557) {
-  dbname = other557.dbname;
-  tablename = other557.tablename;
-  partitionname = other557.partitionname;
-  type = other557.type;
-  state = other557.state;
-  workerid = other557.workerid;
-  start = other557.start;
-  runAs = other557.runAs;
-  hightestTxnId = other557.hightestTxnId;
-  metaInfo = other557.metaInfo;
-  endTime = other557.endTime;
-  hadoopJobId = other557.hadoopJobId;
-  __isset = other557.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other558) {
-  dbname = other558.dbname;
-  tablename = other558.tablename;
-  partitionname = other558.partitionname;
-  type = other558.type;
-  state = other558.state;
-  workerid = other558.workerid;
-  start = other558.start;
-  runAs = other558.runAs;
-  hightestTxnId = other558.hightestTxnId;
-  metaInfo = other558.metaInfo;
-  endTime = other558.endTime;
-  hadoopJobId = other558.hadoopJobId;
-  __isset = other558.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other565) {
+  dbname = other565.dbname;
+  tablename = other565.tablename;
+  partitionname = other565.partitionname;
+  type = other565.type;
+  state = other565.state;
+  workerid = other565.workerid;
+  start = other565.start;
+  runAs = other565.runAs;
+  hightestTxnId = other565.hightestTxnId;
+  metaInfo = other565.metaInfo;
+  endTime = other565.endTime;
+  hadoopJobId = other565.hadoopJobId;
+  __isset = other565.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other566) {
+  dbname = other566.dbname;
+  tablename = other566.tablename;
+  partitionname = other566.partitionname;
+  type = other566.type;
+  state = other566.state;
+  workerid = other566.workerid;
+  start = other566.start;
+  runAs = other566.runAs;
+  hightestTxnId = other566.hightestTxnId;
+  metaInfo = other566.metaInfo;
+  endTime = other566.endTime;
+  hadoopJobId = other566.hadoopJobId;
+  __isset = other566.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -13251,14 +13357,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size559;
-            ::apache::thrift::protocol::TType _etype562;
-            xfer += iprot->readListBegin(_etype562, _size559);
-            this->compacts.resize(_size559);
-            uint32_t _i563;
-            for (_i563 = 0; _i563 < _size559; ++_i563)
+            uint32_t _size567;
+            ::apache::thrift::protocol::TType _etype570;
+            xfer += iprot->readListBegin(_etype570, _size567);
+            this->compacts.resize(_size567);
+            uint32_t _i571;
+            for (_i571 = 0; _i571 < _size567; ++_i571)
             {
-              xfer += this->compacts[_i563].read(iprot);
+              xfer += this->compacts[_i571].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13289,10 +13395,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter564;
-    for (_iter564 = this->compacts.begin(); _iter564 != this->compacts.end(); ++_iter564)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter572;
+    for (_iter572 = this->compacts.begin(); _iter572 != this->compacts.end(); ++_iter572)
     {
-      xfer += (*_iter564).write(oprot);
+      xfer += (*_iter572).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13308,11 +13414,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other565) {
-  compacts = other565.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other573) {
+  compacts = other573.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other566) {
-  compacts = other566.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other574) {
+  compacts = other574.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -13396,14 +13502,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size567;
-            ::apache::thrift::protocol::TType _etype570;
-            xfer += iprot->readListBegin(_etype570, _size567);
-            this->partitionnames.resize(_size567);
-            uint32_t _i571;
-            for (_i571 = 0; _i571 < _size567; ++_i571)
+            uint32_t _size575;
+            ::apache::thrift::protocol::TType _etype578;
+            xfer += iprot->readListBegin(_etype578, _size575);
+            this->partitionnames.resize(_size575);
+            uint32_t _i579;
+            for (_i579 = 0; _i579 < _size575; ++_i579)
             {
-              xfer += iprot->readString(this->partitionnames[_i571]);
+              xfer += iprot->readString(this->partitionnames[_i579]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13452,10 +13558,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter572;
-    for (_iter572 = this->partitionnames.begin(); _iter572 != this->partitionnames.end(); ++_iter572)
+    std::vector<std::string> ::const_iterator _iter580;
+    for (_iter580 = this->partitionnames.begin(); _iter580 != this->partitionnames.end(); ++_iter580)
     {
-      xfer += oprot->writeString((*_iter572));
+      xfer += oprot->writeString((*_iter580));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13474,17 +13580,17 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.partitionnames, b.partitionnames);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other573) {
-  txnid = other573.txnid;
-  dbname = other573.dbname;
-  tablename = other573.tablename;
-  partitionnames = other573.partitionnames;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other581) {
+  txnid = other581.txnid;
+  dbname = other581.dbname;
+  tablename = other581.tablename;
+  partitionnames = other581.partitionnames;
 }
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other574) {
-  txnid = other574.txnid;
-  dbname = other574.dbname;
-  tablename = other574.tablename;
-  partitionnames = other574.partitionnames;
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other582) {
+  txnid = other582.txnid;
+  dbname = other582.dbname;
+  tablename = other582.tablename;
+  partitionnames = other582.partitionnames;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -13589,15 +13695,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other575) {
-  lastEvent = other575.lastEvent;
-  maxEvents = other575.maxEvents;
-  __isset = other575.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other583) {
+  lastEvent = other583.lastEvent;
+  maxEvents = other583.maxEvents;
+  __isset = other583.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other576) {
-  lastEvent = other576.lastEvent;
-  maxEvents = other576.maxEvents;
-  __isset = other576.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other584) {
+  lastEvent = other584.lastEvent;
+  maxEvents = other584.maxEvents;
+  __isset = other584.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -13779,23 +13885,23 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other577) {
-  eventId = other577.eventId;
-  eventTime = other577.eventTime;
-  eventType = other577.eventType;
-  dbName = other577.dbName;
-  tableName = other577.tableName;
-  message = other577.message;
-  __isset = other577.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other578) {
-  eventId = other578.eventId;
-  eventTime = other578.eventTime;
-  eventType = other578.eventType;
-  dbName = other578.dbName;
-  tableName = other578.tableName;
-  message = other578.message;
-  __isset = other578.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other585) {
+  eventId = other585.eventId;
+  eventTime = other585.eventTime;
+  eventType = other585.eventType;
+  dbName = other585.dbName;
+  tableName = other585.tableName;
+  message = other585.message;
+  __isset = other585.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other586) {
+  eventId = other586.eventId;
+  eventTime = other586.eventTime;
+  eventType = other586.eventType;
+  dbName = other586.dbName;
+  tableName = other586.tableName;
+  message = other586.message;
+  __isset = other586.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -13845,14 +13951,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size579;
-            ::apache::thrift::protocol::TType _etype582;
-            xfer += iprot->readListBegin(_etype582, _size579);
-            this->events.resize(_size579);
-            uint32_t _i583;
-            for (_i583 = 0; _i583 < _size579; ++_i583)
+            uint32_t _size587;
+            ::apache::thrift::protocol::TType _etype590;
+            xfer += iprot->readListBegin(_etype590, _size587);
+            this->events.resize(_size587);
+            uint32_t _i591;
+            for (_i591 = 0; _i591 < _size587; ++_i591)
             {
-              xfer += this->events[_i583].read(iprot);
+              xfer += this->events[_i591].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13883,10 +13989,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter584;
-    for (_iter584 = this->events.begin(); _iter584 != this->events.end(); ++_iter584)
+    std::vector<NotificationEvent> ::const_iterator _iter592;
+    for (_iter592 = this->events.begin(); _iter592 != this->events.end(); ++_iter592)
     {
-      xfer += (*_iter584).write(oprot);
+      xfer += (*_iter592).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13902,11 +14008,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other585) {
-  events = other585.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other593) {
+  events = other593.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other586) {
-  events = other586.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other594) {
+  events = other594.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -13988,11 +14094,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other587) {
-  eventId = other587.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other595) {
+  eventId = other595.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other588) {
-  eventId = other588.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other596) {
+  eventId = other596.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -14037,14 +14143,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size589;
-            ::apache::thrift::protocol::TType _etype592;
-            xfer += iprot->readListBegin(_etype592, _size589);
-            this->filesAdded.resize(_size589);
-            uint32_t _i593;
-            for (_i593 = 0; _i593 < _size589; ++_i593)
+            uint32_t _size597;
+            ::apache::thrift::protocol::TType _etype600;
+            xfer += iprot->readListBegin(_etype600, _size597);
+            this->filesAdded.resize(_size597);
+            uint32_t _i601;
+            for (_i601 = 0; _i601 < _size597; ++_i601)
             {
-              xfer += iprot->readString(this->filesAdded[_i593]);
+              xfer += iprot->readString(this->filesAdded[_i601]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14075,10 +14181,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter594;
-    for (_iter594 = this->filesAdded.begin(); _iter594 != this->filesAdded.end(); ++_iter594)
+    std::vector<std::string> ::const_iterator _iter602;
+    for (_iter602 = this->filesAdded.begin(); _iter602 != this->filesAdded.end(); ++_iter602)
     {
-      xfer += oprot->writeString((*_iter594));
+      xfer += oprot->writeString((*_iter602));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14094,11 +14200,11 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.filesAdded, b.filesAdded);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other595) {
-  filesAdded = other595.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other603) {
+  filesAdded = other603.filesAdded;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other596) {
-  filesAdded = other596.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other604) {
+  filesAdded = other604.filesAdded;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -14178,13 +14284,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other597) {
-  insertData = other597.insertData;
-  __isset = other597.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other605) {
+  insertData = other605.insertData;
+  __isset = other605.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other598) {
-  insertData = other598.insertData;
-  __isset = other598.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other606) {
+  insertData = other606.insertData;
+  __isset = other606.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -14281,14 +14387,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size599;
-            ::apache::thrift::protocol::TType _etype602;
-            xfer += iprot->readListBegin(_etype602, _size599);
-            this->partitionVals.resize(_size599);
-            uint32_t _i603;
-            for (_i603 = 0; _i603 < _size599; ++_i603)
+            uint32_t _size607;
+            ::apache::thrift::protocol::TType _etype610;
+            xfer += iprot->readListBegin(_etype610, _size607);
+            this->partitionVals.resize(_size607);
+            uint32_t _i611;
+            for (_i611 = 0; _i611 < _size607; ++_i611)
             {
-              xfer += iprot->readString(this->partitionVals[_i603]);
+              xfer += iprot->readString(this->partitionVals[_i611]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14340,10 +14446,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 _iter604;
-      for (_iter604 = this->partitionVals.begin(); _iter604 != this->partitionVals.end(); ++_iter604)
+      std::vector<std::string> ::const_iterator _iter612;
+      for (_iter612 = this->partitionVals.begin(); _iter612 != this->partitionVals.end(); ++_iter612)
       {
-        xfer += oprot->writeString((*_iter604));
+        xfer += oprot->writeString((*_iter612));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14364,21 +14470,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other605) {
-  successful = other605.successful;
-  data = other605.data;
-  dbName = other605.dbName;
-  tableName = other605.tableName;
-  partitionVals = other605.partitionVals;
-  __isset = other605.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other606) {
-  successful = other606.successful;
-  data = other606.data;
-  dbName = other606.dbName;
-  tableName = other606.tableName;
-  partitionVals = other606.partitionVals;
-  __isset = other606.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other613) {
+  successful = other613.successful;
+  data = other613.data;
+  dbName = other613.dbName;
+  tableName = other613.tableName;
+  partitionVals = other613.partitionVals;
+  __isset = other613.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other614) {
+  successful = other614.successful;
+  data = other614.data;
+  dbName = other614.dbName;
+  tableName = other614.tableName;
+  partitionVals = other614.partitionVals;
+  __isset = other614.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -14441,11 +14547,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other607) {
-  (void) other607;
+FireEventResponse::FireEventResponse(const FireEventResponse& other615) {
+  (void) other615;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other608) {
-  (void) other608;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other616) {
+  (void) other616;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -14489,14 +14595,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size609;
-            ::apache::thrift::protocol::TType _etype612;
-            xfer += iprot->readListBegin(_etype612, _size609);
-            this->functions.resize(_size609);
-            uint32_t _i613;
-            for (_i613 = 0; _i613 < _size609; ++_i613)
+            uint32_t _size617;
+            ::apache::thrift::protocol::TType _etype620;
+            xfer += iprot->readListBegin(_etype620, _size617);
+            this->functions.resize(_size617);
+            uint32_t _i621;
+            for (_i621 = 0; _i621 < _size617; ++_i621)
             {
-              xfer += this->functions[_i613].read(iprot);
+              xfer += this->functions[_i621].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14526,10 +14632,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 _iter614;
-      for (_iter614 = this->functions.begin(); _iter614 != this->functions.end(); ++_iter614)
+      std::vector<Function> ::const_iterator _iter622;
+      for (_iter622 = this->functions.begin(); _iter622 != this->functions.end(); ++_iter622)
       {
-        xfer += (*_iter614).write(oprot);
+        xfer += (*_iter622).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14546,13 +14652,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other615) {
-  functions = other615.functions;
-  __isset = other615.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other623) {
+  functions = other623.functions;
+  __isset = other623.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other616) {
-  functions = other616.functions;
-  __isset = other616.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other624) {
+  functions = other624.functions;
+  __isset = other624.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -14632,13 +14738,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other617) : TException() {
-  message = other617.message;
-  __isset = other617.__isset;
+MetaException::MetaException(const MetaException& other625) : TException() {
+  message = other625.message;
+  __isset = other625.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other618) {
-  message = other618.message;
-  __isset = other618.__isset;
+MetaException& MetaException::operator=(const MetaException& other626) {
+  message = other626.message;
+  __isset = other626.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -14729,13 +14835,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other619) : TException() {
-  message = other619.message;
-  __isset = other619.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other627) : TException() {
+  message = other627.message;
+  __isset = other627.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other620) {
-  message = other620.message;
-  __isset = other620.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other628) {
+  message = other628.message;
+  __isset = other628.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -14826,13 +14932,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other621) : TException() {
-  message = other621.message;
-  __isset = other621.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other629) : TException() {
+  message = other629.message;
+  __isset = other629.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other622) {
-  message = other622.message;
-  __isset = other622.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other630) {
+  message = other630.message;
+  __isset = other630.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -14923,13 +15029,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other623) : TException() {
-  message = other623.message;
-  __isset = other623.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other631) : TException() {
+  message = other631.message;
+  __isset = other631.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other624) {
-  message = other624.message;
-  __isset = other624.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other632) {
+  message = other632.message;
+  __isset = other632.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15020,13 +15126,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other625) : TException() {
-  message = other625.message;
-  __isset = other625.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other633) : TException() {
+  message = other633.message;
+  __isset = other633.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other626) {
-  message = other626.message;
-  __isset = other626.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other634) {
+  message = other634.message;
+  __isset = other634.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15117,13 +15223,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other627) : TException() {
-  message = other627.message;
-  __isset = other627.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other635) : TException() {
+  message = other635.message;
+  __isset = other635.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other628) {
-  message = other628.message;
-  __isset = other628.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other636) {
+  message = other636.message;
+  __isset = other636.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15214,13 +15320,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other629) : TException() {
-  message = other629.message;
-  __isset = other629.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other637) : TException() {
+  message = other637.message;
+  __isset = other637.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other630) {
-  message = other630.message;
-  __isset = other630.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other638) {
+  message = other638.message;
+  __isset = other638.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -15311,13 +15417,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other631) : TException() {
-  message = other631.message;
-  __isset = other631.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other639) : TException() {
+  message = other639.message;
+  __isset = other639.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other632) {
-  message = other632.message;
-  __isset = other632.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other640) {
+  message = other640.message;
+  __isset = other640.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -15408,13 +15514,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other633) : TException() {
-  message = other633.message;
-  __isset = other633.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other641) : TException() {
+  message = other641.message;
+  __isset = other641.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other634) {
-  message = other634.message;
-  __isset = other634.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other642) {
+  message = other642.message;
+  __isset = other642.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -15505,13 +15611,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other635) : TException() {
-  message = other635.message;
-  __isset = other635.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other643) : TException() {
+  message = other643.message;
+  __isset = other643.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other636) {
-  message = other636.message;
-  __isset = other636.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other644) {
+  message = other644.message;
+  __isset = other644.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -15602,13 +15708,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other637) : TException() {
-  message = other637.message;
-  __isset = other637.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other645) : TException() {
+  message = other645.message;
+  __isset = other645.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other638) {
-  message = other638.message;
-  __isset = other638.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other646) {
+  message = other646.message;
+  __isset = other646.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -15699,13 +15805,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other639) : TException() {
-  message = other639.message;
-  __isset = other639.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other647) : TException() {
+  message = other647.message;
+  __isset = other647.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other640) {
-  message = other640.message;
-  __isset = other640.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other648) {
+  message = other648.message;
+  __isset = other648.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -15796,13 +15902,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other641) : TException() {
-  message = other641.message;
-  __isset = other641.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other649) : TException() {
+  message = other649.message;
+  __isset = other649.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other642) {
-  message = other642.message;
-  __isset = other642.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other650) {
+  message = other650.message;
+  __isset = other650.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -15893,13 +15999,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other643) : TException() {
-  message = other643.message;
-  __isset = other643.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other651) : TException() {
+  message = other651.message;
+  __isset = other651.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other644) {
-  message = other644.message;
-  __isset = other644.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other652) {
+  message = other652.message;
+  __isset = other652.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -15990,13 +16096,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other645) : TException() {
-  message = other645.message;
-  __isset = other645.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other653) : TException() {
+  message = other653.message;
+  __isset = other653.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other646) {
-  message = other646.message;
-  __isset = other646.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other654) {
+  message = other654.message;
+  __isset = other654.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16087,13 +16193,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other647) : TException() {
-  message = other647.message;
-  __isset = other647.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other655) : TException() {
+  message = other655.message;
+  __isset = other655.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other648) {
-  message = other648.message;
-  __isset = other648.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other656) {
+  message = other656.message;
+  __isset = other656.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 7039a75..fbe99c1 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -271,6 +271,8 @@ class OpenTxnsResponse;
 
 class AbortTxnRequest;
 
+class AbortTxnsRequest;
+
 class CommitTxnRequest;
 
 class LockComponent;
@@ -4390,6 +4392,46 @@ inline std::ostream& operator<<(std::ostream& out, const AbortTxnRequest& obj)
 }
 
 
+class AbortTxnsRequest {
+ public:
+
+  AbortTxnsRequest(const AbortTxnsRequest&);
+  AbortTxnsRequest& operator=(const AbortTxnsRequest&);
+  AbortTxnsRequest() {
+  }
+
+  virtual ~AbortTxnsRequest() throw();
+  std::vector<int64_t>  txn_ids;
+
+  void __set_txn_ids(const std::vector<int64_t> & val);
+
+  bool operator == (const AbortTxnsRequest & rhs) const
+  {
+    if (!(txn_ids == rhs.txn_ids))
+      return false;
+    return true;
+  }
+  bool operator != (const AbortTxnsRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const AbortTxnsRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(AbortTxnsRequest &a, AbortTxnsRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class CommitTxnRequest {
  public:
 


[7/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 5efda4f..176b634 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 _size649;
-            ::apache::thrift::protocol::TType _etype652;
-            xfer += iprot->readListBegin(_etype652, _size649);
-            this->success.resize(_size649);
-            uint32_t _i653;
-            for (_i653 = 0; _i653 < _size649; ++_i653)
+            uint32_t _size657;
+            ::apache::thrift::protocol::TType _etype660;
+            xfer += iprot->readListBegin(_etype660, _size657);
+            this->success.resize(_size657);
+            uint32_t _i661;
+            for (_i661 = 0; _i661 < _size657; ++_i661)
             {
-              xfer += iprot->readString(this->success[_i653]);
+              xfer += iprot->readString(this->success[_i661]);
             }
             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 _iter654;
-      for (_iter654 = this->success.begin(); _iter654 != this->success.end(); ++_iter654)
+      std::vector<std::string> ::const_iterator _iter662;
+      for (_iter662 = this->success.begin(); _iter662 != this->success.end(); ++_iter662)
       {
-        xfer += oprot->writeString((*_iter654));
+        xfer += oprot->writeString((*_iter662));
       }
       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 _size655;
-            ::apache::thrift::protocol::TType _etype658;
-            xfer += iprot->readListBegin(_etype658, _size655);
-            (*(this->success)).resize(_size655);
-            uint32_t _i659;
-            for (_i659 = 0; _i659 < _size655; ++_i659)
+            uint32_t _size663;
+            ::apache::thrift::protocol::TType _etype666;
+            xfer += iprot->readListBegin(_etype666, _size663);
+            (*(this->success)).resize(_size663);
+            uint32_t _i667;
+            for (_i667 = 0; _i667 < _size663; ++_i667)
             {
-              xfer += iprot->readString((*(this->success))[_i659]);
+              xfer += iprot->readString((*(this->success))[_i667]);
             }
             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 _size660;
-            ::apache::thrift::protocol::TType _etype663;
-            xfer += iprot->readListBegin(_etype663, _size660);
-            this->success.resize(_size660);
-            uint32_t _i664;
-            for (_i664 = 0; _i664 < _size660; ++_i664)
+            uint32_t _size668;
+            ::apache::thrift::protocol::TType _etype671;
+            xfer += iprot->readListBegin(_etype671, _size668);
+            this->success.resize(_size668);
+            uint32_t _i672;
+            for (_i672 = 0; _i672 < _size668; ++_i672)
             {
-              xfer += iprot->readString(this->success[_i664]);
+              xfer += iprot->readString(this->success[_i672]);
             }
             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 _iter665;
-      for (_iter665 = this->success.begin(); _iter665 != this->success.end(); ++_iter665)
+      std::vector<std::string> ::const_iterator _iter673;
+      for (_iter673 = this->success.begin(); _iter673 != this->success.end(); ++_iter673)
       {
-        xfer += oprot->writeString((*_iter665));
+        xfer += oprot->writeString((*_iter673));
       }
       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 _size666;
-            ::apache::thrift::protocol::TType _etype669;
-            xfer += iprot->readListBegin(_etype669, _size666);
-            (*(this->success)).resize(_size666);
-            uint32_t _i670;
-            for (_i670 = 0; _i670 < _size666; ++_i670)
+            uint32_t _size674;
+            ::apache::thrift::protocol::TType _etype677;
+            xfer += iprot->readListBegin(_etype677, _size674);
+            (*(this->success)).resize(_size674);
+            uint32_t _i678;
+            for (_i678 = 0; _i678 < _size674; ++_i678)
             {
-              xfer += iprot->readString((*(this->success))[_i670]);
+              xfer += iprot->readString((*(this->success))[_i678]);
             }
             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 _size671;
-            ::apache::thrift::protocol::TType _ktype672;
-            ::apache::thrift::protocol::TType _vtype673;
-            xfer += iprot->readMapBegin(_ktype672, _vtype673, _size671);
-            uint32_t _i675;
-            for (_i675 = 0; _i675 < _size671; ++_i675)
+            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)
             {
-              std::string _key676;
-              xfer += iprot->readString(_key676);
-              Type& _val677 = this->success[_key676];
-              xfer += _val677.read(iprot);
+              std::string _key684;
+              xfer += iprot->readString(_key684);
+              Type& _val685 = this->success[_key684];
+              xfer += _val685.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 _iter678;
-      for (_iter678 = this->success.begin(); _iter678 != this->success.end(); ++_iter678)
+      std::map<std::string, Type> ::const_iterator _iter686;
+      for (_iter686 = this->success.begin(); _iter686 != this->success.end(); ++_iter686)
       {
-        xfer += oprot->writeString(_iter678->first);
-        xfer += _iter678->second.write(oprot);
+        xfer += oprot->writeString(_iter686->first);
+        xfer += _iter686->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 _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)
+            uint32_t _size687;
+            ::apache::thrift::protocol::TType _ktype688;
+            ::apache::thrift::protocol::TType _vtype689;
+            xfer += iprot->readMapBegin(_ktype688, _vtype689, _size687);
+            uint32_t _i691;
+            for (_i691 = 0; _i691 < _size687; ++_i691)
             {
-              std::string _key684;
-              xfer += iprot->readString(_key684);
-              Type& _val685 = (*(this->success))[_key684];
-              xfer += _val685.read(iprot);
+              std::string _key692;
+              xfer += iprot->readString(_key692);
+              Type& _val693 = (*(this->success))[_key692];
+              xfer += _val693.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 _size686;
-            ::apache::thrift::protocol::TType _etype689;
-            xfer += iprot->readListBegin(_etype689, _size686);
-            this->success.resize(_size686);
-            uint32_t _i690;
-            for (_i690 = 0; _i690 < _size686; ++_i690)
+            uint32_t _size694;
+            ::apache::thrift::protocol::TType _etype697;
+            xfer += iprot->readListBegin(_etype697, _size694);
+            this->success.resize(_size694);
+            uint32_t _i698;
+            for (_i698 = 0; _i698 < _size694; ++_i698)
             {
-              xfer += this->success[_i690].read(iprot);
+              xfer += this->success[_i698].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 _iter691;
-      for (_iter691 = this->success.begin(); _iter691 != this->success.end(); ++_iter691)
+      std::vector<FieldSchema> ::const_iterator _iter699;
+      for (_iter699 = this->success.begin(); _iter699 != this->success.end(); ++_iter699)
       {
-        xfer += (*_iter691).write(oprot);
+        xfer += (*_iter699).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 _size692;
-            ::apache::thrift::protocol::TType _etype695;
-            xfer += iprot->readListBegin(_etype695, _size692);
-            (*(this->success)).resize(_size692);
-            uint32_t _i696;
-            for (_i696 = 0; _i696 < _size692; ++_i696)
+            uint32_t _size700;
+            ::apache::thrift::protocol::TType _etype703;
+            xfer += iprot->readListBegin(_etype703, _size700);
+            (*(this->success)).resize(_size700);
+            uint32_t _i704;
+            for (_i704 = 0; _i704 < _size700; ++_i704)
             {
-              xfer += (*(this->success))[_i696].read(iprot);
+              xfer += (*(this->success))[_i704].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 _size697;
-            ::apache::thrift::protocol::TType _etype700;
-            xfer += iprot->readListBegin(_etype700, _size697);
-            this->success.resize(_size697);
-            uint32_t _i701;
-            for (_i701 = 0; _i701 < _size697; ++_i701)
+            uint32_t _size705;
+            ::apache::thrift::protocol::TType _etype708;
+            xfer += iprot->readListBegin(_etype708, _size705);
+            this->success.resize(_size705);
+            uint32_t _i709;
+            for (_i709 = 0; _i709 < _size705; ++_i709)
             {
-              xfer += this->success[_i701].read(iprot);
+              xfer += this->success[_i709].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 _iter702;
-      for (_iter702 = this->success.begin(); _iter702 != this->success.end(); ++_iter702)
+      std::vector<FieldSchema> ::const_iterator _iter710;
+      for (_iter710 = this->success.begin(); _iter710 != this->success.end(); ++_iter710)
       {
-        xfer += (*_iter702).write(oprot);
+        xfer += (*_iter710).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 _size703;
-            ::apache::thrift::protocol::TType _etype706;
-            xfer += iprot->readListBegin(_etype706, _size703);
-            (*(this->success)).resize(_size703);
-            uint32_t _i707;
-            for (_i707 = 0; _i707 < _size703; ++_i707)
+            uint32_t _size711;
+            ::apache::thrift::protocol::TType _etype714;
+            xfer += iprot->readListBegin(_etype714, _size711);
+            (*(this->success)).resize(_size711);
+            uint32_t _i715;
+            for (_i715 = 0; _i715 < _size711; ++_i715)
             {
-              xfer += (*(this->success))[_i707].read(iprot);
+              xfer += (*(this->success))[_i715].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 _size708;
-            ::apache::thrift::protocol::TType _etype711;
-            xfer += iprot->readListBegin(_etype711, _size708);
-            this->success.resize(_size708);
-            uint32_t _i712;
-            for (_i712 = 0; _i712 < _size708; ++_i712)
+            uint32_t _size716;
+            ::apache::thrift::protocol::TType _etype719;
+            xfer += iprot->readListBegin(_etype719, _size716);
+            this->success.resize(_size716);
+            uint32_t _i720;
+            for (_i720 = 0; _i720 < _size716; ++_i720)
             {
-              xfer += this->success[_i712].read(iprot);
+              xfer += this->success[_i720].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 _iter713;
-      for (_iter713 = this->success.begin(); _iter713 != this->success.end(); ++_iter713)
+      std::vector<FieldSchema> ::const_iterator _iter721;
+      for (_iter721 = this->success.begin(); _iter721 != this->success.end(); ++_iter721)
       {
-        xfer += (*_iter713).write(oprot);
+        xfer += (*_iter721).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 _size714;
-            ::apache::thrift::protocol::TType _etype717;
-            xfer += iprot->readListBegin(_etype717, _size714);
-            (*(this->success)).resize(_size714);
-            uint32_t _i718;
-            for (_i718 = 0; _i718 < _size714; ++_i718)
+            uint32_t _size722;
+            ::apache::thrift::protocol::TType _etype725;
+            xfer += iprot->readListBegin(_etype725, _size722);
+            (*(this->success)).resize(_size722);
+            uint32_t _i726;
+            for (_i726 = 0; _i726 < _size722; ++_i726)
             {
-              xfer += (*(this->success))[_i718].read(iprot);
+              xfer += (*(this->success))[_i726].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 _size719;
-            ::apache::thrift::protocol::TType _etype722;
-            xfer += iprot->readListBegin(_etype722, _size719);
-            this->success.resize(_size719);
-            uint32_t _i723;
-            for (_i723 = 0; _i723 < _size719; ++_i723)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            this->success.resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += this->success[_i723].read(iprot);
+              xfer += this->success[_i731].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 _iter724;
-      for (_iter724 = this->success.begin(); _iter724 != this->success.end(); ++_iter724)
+      std::vector<FieldSchema> ::const_iterator _iter732;
+      for (_iter732 = this->success.begin(); _iter732 != this->success.end(); ++_iter732)
       {
-        xfer += (*_iter724).write(oprot);
+        xfer += (*_iter732).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 _size725;
-            ::apache::thrift::protocol::TType _etype728;
-            xfer += iprot->readListBegin(_etype728, _size725);
-            (*(this->success)).resize(_size725);
-            uint32_t _i729;
-            for (_i729 = 0; _i729 < _size725; ++_i729)
+            uint32_t _size733;
+            ::apache::thrift::protocol::TType _etype736;
+            xfer += iprot->readListBegin(_etype736, _size733);
+            (*(this->success)).resize(_size733);
+            uint32_t _i737;
+            for (_i737 = 0; _i737 < _size733; ++_i737)
             {
-              xfer += (*(this->success))[_i729].read(iprot);
+              xfer += (*(this->success))[_i737].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size730;
-            ::apache::thrift::protocol::TType _etype733;
-            xfer += iprot->readListBegin(_etype733, _size730);
-            this->success.resize(_size730);
-            uint32_t _i734;
-            for (_i734 = 0; _i734 < _size730; ++_i734)
+            uint32_t _size738;
+            ::apache::thrift::protocol::TType _etype741;
+            xfer += iprot->readListBegin(_etype741, _size738);
+            this->success.resize(_size738);
+            uint32_t _i742;
+            for (_i742 = 0; _i742 < _size738; ++_i742)
             {
-              xfer += iprot->readString(this->success[_i734]);
+              xfer += iprot->readString(this->success[_i742]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter735;
-      for (_iter735 = this->success.begin(); _iter735 != this->success.end(); ++_iter735)
+      std::vector<std::string> ::const_iterator _iter743;
+      for (_iter743 = this->success.begin(); _iter743 != this->success.end(); ++_iter743)
       {
-        xfer += oprot->writeString((*_iter735));
+        xfer += oprot->writeString((*_iter743));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _etype739;
-            xfer += iprot->readListBegin(_etype739, _size736);
-            (*(this->success)).resize(_size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size744;
+            ::apache::thrift::protocol::TType _etype747;
+            xfer += iprot->readListBegin(_etype747, _size744);
+            (*(this->success)).resize(_size744);
+            uint32_t _i748;
+            for (_i748 = 0; _i748 < _size744; ++_i748)
             {
-              xfer += iprot->readString((*(this->success))[_i740]);
+              xfer += iprot->readString((*(this->success))[_i748]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5338,14 +5338,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size741;
-            ::apache::thrift::protocol::TType _etype744;
-            xfer += iprot->readListBegin(_etype744, _size741);
-            this->success.resize(_size741);
-            uint32_t _i745;
-            for (_i745 = 0; _i745 < _size741; ++_i745)
+            uint32_t _size749;
+            ::apache::thrift::protocol::TType _etype752;
+            xfer += iprot->readListBegin(_etype752, _size749);
+            this->success.resize(_size749);
+            uint32_t _i753;
+            for (_i753 = 0; _i753 < _size749; ++_i753)
             {
-              xfer += iprot->readString(this->success[_i745]);
+              xfer += iprot->readString(this->success[_i753]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5384,10 +5384,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter746;
-      for (_iter746 = this->success.begin(); _iter746 != this->success.end(); ++_iter746)
+      std::vector<std::string> ::const_iterator _iter754;
+      for (_iter754 = this->success.begin(); _iter754 != this->success.end(); ++_iter754)
       {
-        xfer += oprot->writeString((*_iter746));
+        xfer += oprot->writeString((*_iter754));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5432,14 +5432,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size747;
-            ::apache::thrift::protocol::TType _etype750;
-            xfer += iprot->readListBegin(_etype750, _size747);
-            (*(this->success)).resize(_size747);
-            uint32_t _i751;
-            for (_i751 = 0; _i751 < _size747; ++_i751)
+            uint32_t _size755;
+            ::apache::thrift::protocol::TType _etype758;
+            xfer += iprot->readListBegin(_etype758, _size755);
+            (*(this->success)).resize(_size755);
+            uint32_t _i759;
+            for (_i759 = 0; _i759 < _size755; ++_i759)
             {
-              xfer += iprot->readString((*(this->success))[_i751]);
+              xfer += iprot->readString((*(this->success))[_i759]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5749,14 +5749,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size752;
-            ::apache::thrift::protocol::TType _etype755;
-            xfer += iprot->readListBegin(_etype755, _size752);
-            this->tbl_names.resize(_size752);
-            uint32_t _i756;
-            for (_i756 = 0; _i756 < _size752; ++_i756)
+            uint32_t _size760;
+            ::apache::thrift::protocol::TType _etype763;
+            xfer += iprot->readListBegin(_etype763, _size760);
+            this->tbl_names.resize(_size760);
+            uint32_t _i764;
+            for (_i764 = 0; _i764 < _size760; ++_i764)
             {
-              xfer += iprot->readString(this->tbl_names[_i756]);
+              xfer += iprot->readString(this->tbl_names[_i764]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5789,10 +5789,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter757;
-    for (_iter757 = this->tbl_names.begin(); _iter757 != this->tbl_names.end(); ++_iter757)
+    std::vector<std::string> ::const_iterator _iter765;
+    for (_iter765 = this->tbl_names.begin(); _iter765 != this->tbl_names.end(); ++_iter765)
     {
-      xfer += oprot->writeString((*_iter757));
+      xfer += oprot->writeString((*_iter765));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5820,10 +5820,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter758;
-    for (_iter758 = (*(this->tbl_names)).begin(); _iter758 != (*(this->tbl_names)).end(); ++_iter758)
+    std::vector<std::string> ::const_iterator _iter766;
+    for (_iter766 = (*(this->tbl_names)).begin(); _iter766 != (*(this->tbl_names)).end(); ++_iter766)
     {
-      xfer += oprot->writeString((*_iter758));
+      xfer += oprot->writeString((*_iter766));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5864,14 +5864,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size759;
-            ::apache::thrift::protocol::TType _etype762;
-            xfer += iprot->readListBegin(_etype762, _size759);
-            this->success.resize(_size759);
-            uint32_t _i763;
-            for (_i763 = 0; _i763 < _size759; ++_i763)
+            uint32_t _size767;
+            ::apache::thrift::protocol::TType _etype770;
+            xfer += iprot->readListBegin(_etype770, _size767);
+            this->success.resize(_size767);
+            uint32_t _i771;
+            for (_i771 = 0; _i771 < _size767; ++_i771)
             {
-              xfer += this->success[_i763].read(iprot);
+              xfer += this->success[_i771].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5926,10 +5926,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter764;
-      for (_iter764 = this->success.begin(); _iter764 != this->success.end(); ++_iter764)
+      std::vector<Table> ::const_iterator _iter772;
+      for (_iter772 = this->success.begin(); _iter772 != this->success.end(); ++_iter772)
       {
-        xfer += (*_iter764).write(oprot);
+        xfer += (*_iter772).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5982,14 +5982,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size765;
-            ::apache::thrift::protocol::TType _etype768;
-            xfer += iprot->readListBegin(_etype768, _size765);
-            (*(this->success)).resize(_size765);
-            uint32_t _i769;
-            for (_i769 = 0; _i769 < _size765; ++_i769)
+            uint32_t _size773;
+            ::apache::thrift::protocol::TType _etype776;
+            xfer += iprot->readListBegin(_etype776, _size773);
+            (*(this->success)).resize(_size773);
+            uint32_t _i777;
+            for (_i777 = 0; _i777 < _size773; ++_i777)
             {
-              xfer += (*(this->success))[_i769].read(iprot);
+              xfer += (*(this->success))[_i777].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6175,14 +6175,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size770;
-            ::apache::thrift::protocol::TType _etype773;
-            xfer += iprot->readListBegin(_etype773, _size770);
-            this->success.resize(_size770);
-            uint32_t _i774;
-            for (_i774 = 0; _i774 < _size770; ++_i774)
+            uint32_t _size778;
+            ::apache::thrift::protocol::TType _etype781;
+            xfer += iprot->readListBegin(_etype781, _size778);
+            this->success.resize(_size778);
+            uint32_t _i782;
+            for (_i782 = 0; _i782 < _size778; ++_i782)
             {
-              xfer += iprot->readString(this->success[_i774]);
+              xfer += iprot->readString(this->success[_i782]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6237,10 +6237,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter775;
-      for (_iter775 = this->success.begin(); _iter775 != this->success.end(); ++_iter775)
+      std::vector<std::string> ::const_iterator _iter783;
+      for (_iter783 = this->success.begin(); _iter783 != this->success.end(); ++_iter783)
       {
-        xfer += oprot->writeString((*_iter775));
+        xfer += oprot->writeString((*_iter783));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6293,14 +6293,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size776;
-            ::apache::thrift::protocol::TType _etype779;
-            xfer += iprot->readListBegin(_etype779, _size776);
-            (*(this->success)).resize(_size776);
-            uint32_t _i780;
-            for (_i780 = 0; _i780 < _size776; ++_i780)
+            uint32_t _size784;
+            ::apache::thrift::protocol::TType _etype787;
+            xfer += iprot->readListBegin(_etype787, _size784);
+            (*(this->success)).resize(_size784);
+            uint32_t _i788;
+            for (_i788 = 0; _i788 < _size784; ++_i788)
             {
-              xfer += iprot->readString((*(this->success))[_i780]);
+              xfer += iprot->readString((*(this->success))[_i788]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7634,14 +7634,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size781;
-            ::apache::thrift::protocol::TType _etype784;
-            xfer += iprot->readListBegin(_etype784, _size781);
-            this->new_parts.resize(_size781);
-            uint32_t _i785;
-            for (_i785 = 0; _i785 < _size781; ++_i785)
+            uint32_t _size789;
+            ::apache::thrift::protocol::TType _etype792;
+            xfer += iprot->readListBegin(_etype792, _size789);
+            this->new_parts.resize(_size789);
+            uint32_t _i793;
+            for (_i793 = 0; _i793 < _size789; ++_i793)
             {
-              xfer += this->new_parts[_i785].read(iprot);
+              xfer += this->new_parts[_i793].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7670,10 +7670,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter786;
-    for (_iter786 = this->new_parts.begin(); _iter786 != this->new_parts.end(); ++_iter786)
+    std::vector<Partition> ::const_iterator _iter794;
+    for (_iter794 = this->new_parts.begin(); _iter794 != this->new_parts.end(); ++_iter794)
     {
-      xfer += (*_iter786).write(oprot);
+      xfer += (*_iter794).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7697,10 +7697,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter787;
-    for (_iter787 = (*(this->new_parts)).begin(); _iter787 != (*(this->new_parts)).end(); ++_iter787)
+    std::vector<Partition> ::const_iterator _iter795;
+    for (_iter795 = (*(this->new_parts)).begin(); _iter795 != (*(this->new_parts)).end(); ++_iter795)
     {
-      xfer += (*_iter787).write(oprot);
+      xfer += (*_iter795).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7909,14 +7909,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size788;
-            ::apache::thrift::protocol::TType _etype791;
-            xfer += iprot->readListBegin(_etype791, _size788);
-            this->new_parts.resize(_size788);
-            uint32_t _i792;
-            for (_i792 = 0; _i792 < _size788; ++_i792)
+            uint32_t _size796;
+            ::apache::thrift::protocol::TType _etype799;
+            xfer += iprot->readListBegin(_etype799, _size796);
+            this->new_parts.resize(_size796);
+            uint32_t _i800;
+            for (_i800 = 0; _i800 < _size796; ++_i800)
             {
-              xfer += this->new_parts[_i792].read(iprot);
+              xfer += this->new_parts[_i800].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7945,10 +7945,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter793;
-    for (_iter793 = this->new_parts.begin(); _iter793 != this->new_parts.end(); ++_iter793)
+    std::vector<PartitionSpec> ::const_iterator _iter801;
+    for (_iter801 = this->new_parts.begin(); _iter801 != this->new_parts.end(); ++_iter801)
     {
-      xfer += (*_iter793).write(oprot);
+      xfer += (*_iter801).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7972,10 +7972,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter794;
-    for (_iter794 = (*(this->new_parts)).begin(); _iter794 != (*(this->new_parts)).end(); ++_iter794)
+    std::vector<PartitionSpec> ::const_iterator _iter802;
+    for (_iter802 = (*(this->new_parts)).begin(); _iter802 != (*(this->new_parts)).end(); ++_iter802)
     {
-      xfer += (*_iter794).write(oprot);
+      xfer += (*_iter802).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8200,14 +8200,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size795;
-            ::apache::thrift::protocol::TType _etype798;
-            xfer += iprot->readListBegin(_etype798, _size795);
-            this->part_vals.resize(_size795);
-            uint32_t _i799;
-            for (_i799 = 0; _i799 < _size795; ++_i799)
+            uint32_t _size803;
+            ::apache::thrift::protocol::TType _etype806;
+            xfer += iprot->readListBegin(_etype806, _size803);
+            this->part_vals.resize(_size803);
+            uint32_t _i807;
+            for (_i807 = 0; _i807 < _size803; ++_i807)
             {
-              xfer += iprot->readString(this->part_vals[_i799]);
+              xfer += iprot->readString(this->part_vals[_i807]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8244,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter800;
-    for (_iter800 = this->part_vals.begin(); _iter800 != this->part_vals.end(); ++_iter800)
+    std::vector<std::string> ::const_iterator _iter808;
+    for (_iter808 = this->part_vals.begin(); _iter808 != this->part_vals.end(); ++_iter808)
     {
-      xfer += oprot->writeString((*_iter800));
+      xfer += oprot->writeString((*_iter808));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8279,10 +8279,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter801;
-    for (_iter801 = (*(this->part_vals)).begin(); _iter801 != (*(this->part_vals)).end(); ++_iter801)
+    std::vector<std::string> ::const_iterator _iter809;
+    for (_iter809 = (*(this->part_vals)).begin(); _iter809 != (*(this->part_vals)).end(); ++_iter809)
     {
-      xfer += oprot->writeString((*_iter801));
+      xfer += oprot->writeString((*_iter809));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8754,14 +8754,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size802;
-            ::apache::thrift::protocol::TType _etype805;
-            xfer += iprot->readListBegin(_etype805, _size802);
-            this->part_vals.resize(_size802);
-            uint32_t _i806;
-            for (_i806 = 0; _i806 < _size802; ++_i806)
+            uint32_t _size810;
+            ::apache::thrift::protocol::TType _etype813;
+            xfer += iprot->readListBegin(_etype813, _size810);
+            this->part_vals.resize(_size810);
+            uint32_t _i814;
+            for (_i814 = 0; _i814 < _size810; ++_i814)
             {
-              xfer += iprot->readString(this->part_vals[_i806]);
+              xfer += iprot->readString(this->part_vals[_i814]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8806,10 +8806,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter807;
-    for (_iter807 = this->part_vals.begin(); _iter807 != this->part_vals.end(); ++_iter807)
+    std::vector<std::string> ::const_iterator _iter815;
+    for (_iter815 = this->part_vals.begin(); _iter815 != this->part_vals.end(); ++_iter815)
     {
-      xfer += oprot->writeString((*_iter807));
+      xfer += oprot->writeString((*_iter815));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8845,10 +8845,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter808;
-    for (_iter808 = (*(this->part_vals)).begin(); _iter808 != (*(this->part_vals)).end(); ++_iter808)
+    std::vector<std::string> ::const_iterator _iter816;
+    for (_iter816 = (*(this->part_vals)).begin(); _iter816 != (*(this->part_vals)).end(); ++_iter816)
     {
-      xfer += oprot->writeString((*_iter808));
+      xfer += oprot->writeString((*_iter816));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9651,14 +9651,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size809;
-            ::apache::thrift::protocol::TType _etype812;
-            xfer += iprot->readListBegin(_etype812, _size809);
-            this->part_vals.resize(_size809);
-            uint32_t _i813;
-            for (_i813 = 0; _i813 < _size809; ++_i813)
+            uint32_t _size817;
+            ::apache::thrift::protocol::TType _etype820;
+            xfer += iprot->readListBegin(_etype820, _size817);
+            this->part_vals.resize(_size817);
+            uint32_t _i821;
+            for (_i821 = 0; _i821 < _size817; ++_i821)
             {
-              xfer += iprot->readString(this->part_vals[_i813]);
+              xfer += iprot->readString(this->part_vals[_i821]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9703,10 +9703,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter814;
-    for (_iter814 = this->part_vals.begin(); _iter814 != this->part_vals.end(); ++_iter814)
+    std::vector<std::string> ::const_iterator _iter822;
+    for (_iter822 = this->part_vals.begin(); _iter822 != this->part_vals.end(); ++_iter822)
     {
-      xfer += oprot->writeString((*_iter814));
+      xfer += oprot->writeString((*_iter822));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9742,10 +9742,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter815;
-    for (_iter815 = (*(this->part_vals)).begin(); _iter815 != (*(this->part_vals)).end(); ++_iter815)
+    std::vector<std::string> ::const_iterator _iter823;
+    for (_iter823 = (*(this->part_vals)).begin(); _iter823 != (*(this->part_vals)).end(); ++_iter823)
     {
-      xfer += oprot->writeString((*_iter815));
+      xfer += oprot->writeString((*_iter823));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9954,14 +9954,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size816;
-            ::apache::thrift::protocol::TType _etype819;
-            xfer += iprot->readListBegin(_etype819, _size816);
-            this->part_vals.resize(_size816);
-            uint32_t _i820;
-            for (_i820 = 0; _i820 < _size816; ++_i820)
+            uint32_t _size824;
+            ::apache::thrift::protocol::TType _etype827;
+            xfer += iprot->readListBegin(_etype827, _size824);
+            this->part_vals.resize(_size824);
+            uint32_t _i828;
+            for (_i828 = 0; _i828 < _size824; ++_i828)
             {
-              xfer += iprot->readString(this->part_vals[_i820]);
+              xfer += iprot->readString(this->part_vals[_i828]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10014,10 +10014,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter821;
-    for (_iter821 = this->part_vals.begin(); _iter821 != this->part_vals.end(); ++_iter821)
+    std::vector<std::string> ::const_iterator _iter829;
+    for (_iter829 = this->part_vals.begin(); _iter829 != this->part_vals.end(); ++_iter829)
     {
-      xfer += oprot->writeString((*_iter821));
+      xfer += oprot->writeString((*_iter829));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10057,10 +10057,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter822;
-    for (_iter822 = (*(this->part_vals)).begin(); _iter822 != (*(this->part_vals)).end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter830;
+    for (_iter830 = (*(this->part_vals)).begin(); _iter830 != (*(this->part_vals)).end(); ++_iter830)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter830));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11066,14 +11066,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size823;
-            ::apache::thrift::protocol::TType _etype826;
-            xfer += iprot->readListBegin(_etype826, _size823);
-            this->part_vals.resize(_size823);
-            uint32_t _i827;
-            for (_i827 = 0; _i827 < _size823; ++_i827)
+            uint32_t _size831;
+            ::apache::thrift::protocol::TType _etype834;
+            xfer += iprot->readListBegin(_etype834, _size831);
+            this->part_vals.resize(_size831);
+            uint32_t _i835;
+            for (_i835 = 0; _i835 < _size831; ++_i835)
             {
-              xfer += iprot->readString(this->part_vals[_i827]);
+              xfer += iprot->readString(this->part_vals[_i835]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11110,10 +11110,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter828;
-    for (_iter828 = this->part_vals.begin(); _iter828 != this->part_vals.end(); ++_iter828)
+    std::vector<std::string> ::const_iterator _iter836;
+    for (_iter836 = this->part_vals.begin(); _iter836 != this->part_vals.end(); ++_iter836)
     {
-      xfer += oprot->writeString((*_iter828));
+      xfer += oprot->writeString((*_iter836));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11145,10 +11145,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter829;
-    for (_iter829 = (*(this->part_vals)).begin(); _iter829 != (*(this->part_vals)).end(); ++_iter829)
+    std::vector<std::string> ::const_iterator _iter837;
+    for (_iter837 = (*(this->part_vals)).begin(); _iter837 != (*(this->part_vals)).end(); ++_iter837)
     {
-      xfer += oprot->writeString((*_iter829));
+      xfer += oprot->writeString((*_iter837));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11337,17 +11337,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size830;
-            ::apache::thrift::protocol::TType _ktype831;
-            ::apache::thrift::protocol::TType _vtype832;
-            xfer += iprot->readMapBegin(_ktype831, _vtype832, _size830);
-            uint32_t _i834;
-            for (_i834 = 0; _i834 < _size830; ++_i834)
+            uint32_t _size838;
+            ::apache::thrift::protocol::TType _ktype839;
+            ::apache::thrift::protocol::TType _vtype840;
+            xfer += iprot->readMapBegin(_ktype839, _vtype840, _size838);
+            uint32_t _i842;
+            for (_i842 = 0; _i842 < _size838; ++_i842)
             {
-              std::string _key835;
-              xfer += iprot->readString(_key835);
-              std::string& _val836 = this->partitionSpecs[_key835];
-              xfer += iprot->readString(_val836);
+              std::string _key843;
+              xfer += iprot->readString(_key843);
+              std::string& _val844 = this->partitionSpecs[_key843];
+              xfer += iprot->readString(_val844);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11408,11 +11408,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter837;
-    for (_iter837 = this->partitionSpecs.begin(); _iter837 != this->partitionSpecs.end(); ++_iter837)
+    std::map<std::string, std::string> ::const_iterator _iter845;
+    for (_iter845 = this->partitionSpecs.begin(); _iter845 != this->partitionSpecs.end(); ++_iter845)
     {
-      xfer += oprot->writeString(_iter837->first);
-      xfer += oprot->writeString(_iter837->second);
+      xfer += oprot->writeString(_iter845->first);
+      xfer += oprot->writeString(_iter845->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11452,11 +11452,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter838;
-    for (_iter838 = (*(this->partitionSpecs)).begin(); _iter838 != (*(this->partitionSpecs)).end(); ++_iter838)
+    std::map<std::string, std::string> ::const_iterator _iter846;
+    for (_iter846 = (*(this->partitionSpecs)).begin(); _iter846 != (*(this->partitionSpecs)).end(); ++_iter846)
     {
-      xfer += oprot->writeString(_iter838->first);
-      xfer += oprot->writeString(_iter838->second);
+      xfer += oprot->writeString(_iter846->first);
+      xfer += oprot->writeString(_iter846->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11701,17 +11701,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size839;
-            ::apache::thrift::protocol::TType _ktype840;
-            ::apache::thrift::protocol::TType _vtype841;
-            xfer += iprot->readMapBegin(_ktype840, _vtype841, _size839);
-            uint32_t _i843;
-            for (_i843 = 0; _i843 < _size839; ++_i843)
+            uint32_t _size847;
+            ::apache::thrift::protocol::TType _ktype848;
+            ::apache::thrift::protocol::TType _vtype849;
+            xfer += iprot->readMapBegin(_ktype848, _vtype849, _size847);
+            uint32_t _i851;
+            for (_i851 = 0; _i851 < _size847; ++_i851)
             {
-              std::string _key844;
-              xfer += iprot->readString(_key844);
-              std::string& _val845 = this->partitionSpecs[_key844];
-              xfer += iprot->readString(_val845);
+              std::string _key852;
+              xfer += iprot->readString(_key852);
+              std::string& _val853 = this->partitionSpecs[_key852];
+              xfer += iprot->readString(_val853);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11772,11 +11772,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter846;
-    for (_iter846 = this->partitionSpecs.begin(); _iter846 != this->partitionSpecs.end(); ++_iter846)
+    std::map<std::string, std::string> ::const_iterator _iter854;
+    for (_iter854 = this->partitionSpecs.begin(); _iter854 != this->partitionSpecs.end(); ++_iter854)
     {
-      xfer += oprot->writeString(_iter846->first);
-      xfer += oprot->writeString(_iter846->second);
+      xfer += oprot->writeString(_iter854->first);
+      xfer += oprot->writeString(_iter854->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11816,11 +11816,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter847;
-    for (_iter847 = (*(this->partitionSpecs)).begin(); _iter847 != (*(this->partitionSpecs)).end(); ++_iter847)
+    std::map<std::string, std::string> ::const_iterator _iter855;
+    for (_iter855 = (*(this->partitionSpecs)).begin(); _iter855 != (*(this->partitionSpecs)).end(); ++_iter855)
     {
-      xfer += oprot->writeString(_iter847->first);
-      xfer += oprot->writeString(_iter847->second);
+      xfer += oprot->writeString(_iter855->first);
+      xfer += oprot->writeString(_iter855->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11877,14 +11877,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size848;
-            ::apache::thrift::protocol::TType _etype851;
-            xfer += iprot->readListBegin(_etype851, _size848);
-            this->success.resize(_size848);
-            uint32_t _i852;
-            for (_i852 = 0; _i852 < _size848; ++_i852)
+            uint32_t _size856;
+            ::apache::thrift::protocol::TType _etype859;
+            xfer += iprot->readListBegin(_etype859, _size856);
+            this->success.resize(_size856);
+            uint32_t _i860;
+            for (_i860 = 0; _i860 < _size856; ++_i860)
             {
-              xfer += this->success[_i852].read(iprot);
+              xfer += this->success[_i860].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11947,10 +11947,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter853;
-      for (_iter853 = this->success.begin(); _iter853 != this->success.end(); ++_iter853)
+      std::vector<Partition> ::const_iterator _iter861;
+      for (_iter861 = this->success.begin(); _iter861 != this->success.end(); ++_iter861)
       {
-        xfer += (*_iter853).write(oprot);
+        xfer += (*_iter861).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12007,14 +12007,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            (*(this->success)).resize(_size854);
-            uint32_t _i858;
-            for (_i858 = 0; _i858 < _size854; ++_i858)
+            uint32_t _size862;
+            ::apache::thrift::protocol::TType _etype865;
+            xfer += iprot->readListBegin(_etype865, _size862);
+            (*(this->success)).resize(_size862);
+            uint32_t _i866;
+            for (_i866 = 0; _i866 < _size862; ++_i866)
             {
-              xfer += (*(this->success))[_i858].read(iprot);
+              xfer += (*(this->success))[_i866].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12113,14 +12113,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size859;
-            ::apache::thrift::protocol::TType _etype862;
-            xfer += iprot->readListBegin(_etype862, _size859);
-            this->part_vals.resize(_size859);
-            uint32_t _i863;
-            for (_i863 = 0; _i863 < _size859; ++_i863)
+            uint32_t _size867;
+            ::apache::thrift::protocol::TType _etype870;
+            xfer += iprot->readListBegin(_etype870, _size867);
+            this->part_vals.resize(_size867);
+            uint32_t _i871;
+            for (_i871 = 0; _i871 < _size867; ++_i871)
             {
-              xfer += iprot->readString(this->part_vals[_i863]);
+              xfer += iprot->readString(this->part_vals[_i871]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12141,14 +12141,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size864;
-            ::apache::thrift::protocol::TType _etype867;
-            xfer += iprot->readListBegin(_etype867, _size864);
-            this->group_names.resize(_size864);
-            uint32_t _i868;
-            for (_i868 = 0; _i868 < _size864; ++_i868)
+            uint32_t _size872;
+            ::apache::thrift::protocol::TType _etype875;
+            xfer += iprot->readListBegin(_etype875, _size872);
+            this->group_names.resize(_size872);
+            uint32_t _i876;
+            for (_i876 = 0; _i876 < _size872; ++_i876)
             {
-              xfer += iprot->readString(this->group_names[_i868]);
+              xfer += iprot->readString(this->group_names[_i876]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12185,10 +12185,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter869;
-    for (_iter869 = this->part_vals.begin(); _iter869 != this->part_vals.end(); ++_iter869)
+    std::vector<std::string> ::const_iterator _iter877;
+    for (_iter877 = this->part_vals.begin(); _iter877 != this->part_vals.end(); ++_iter877)
     {
-      xfer += oprot->writeString((*_iter869));
+      xfer += oprot->writeString((*_iter877));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12201,10 +12201,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter870;
-    for (_iter870 = this->group_names.begin(); _iter870 != this->group_names.end(); ++_iter870)
+    std::vector<std::string> ::const_iterator _iter878;
+    for (_iter878 = this->group_names.begin(); _iter878 != this->group_names.end(); ++_iter878)
     {
-      xfer += oprot->writeString((*_iter870));
+      xfer += oprot->writeString((*_iter878));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12236,10 +12236,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter871;
-    for (_iter871 = (*(this->part_vals)).begin(); _iter871 != (*(this->part_vals)).end(); ++_iter871)
+    std::vector<std::string> ::const_iterator _iter879;
+    for (_iter879 = (*(this->part_vals)).begin(); _iter879 != (*(this->part_vals)).end(); ++_iter879)
     {
-      xfer += oprot->writeString((*_iter871));
+      xfer += oprot->writeString((*_iter879));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12252,10 +12252,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter872;
-    for (_iter872 = (*(this->group_names)).begin(); _iter872 != (*(this->group_names)).end(); ++_iter872)
+    std::vector<std::string> ::const_iterator _iter880;
+    for (_iter880 = (*(this->group_names)).begin(); _iter880 != (*(this->group_names)).end(); ++_iter880)
     {
-      xfer += oprot->writeString((*_iter872));
+      xfer += oprot->writeString((*_iter880));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12814,14 +12814,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size873;
-            ::apache::thrift::protocol::TType _etype876;
-            xfer += iprot->readListBegin(_etype876, _size873);
-            this->success.resize(_size873);
-            uint32_t _i877;
-            for (_i877 = 0; _i877 < _size873; ++_i877)
+            uint32_t _size881;
+            ::apache::thrift::protocol::TType _etype884;
+            xfer += iprot->readListBegin(_etype884, _size881);
+            this->success.resize(_size881);
+            uint32_t _i885;
+            for (_i885 = 0; _i885 < _size881; ++_i885)
             {
-              xfer += this->success[_i877].read(iprot);
+              xfer += this->success[_i885].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12868,10 +12868,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter878;
-      for (_iter878 = this->success.begin(); _iter878 != this->success.end(); ++_iter878)
+      std::vector<Partition> ::const_iterator _iter886;
+      for (_iter886 = this->success.begin(); _iter886 != this->success.end(); ++_iter886)
       {
-        xfer += (*_iter878).write(oprot);
+        xfer += (*_iter886).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12920,14 +12920,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size879;
-            ::apache::thrift::protocol::TType _etype882;
-            xfer += iprot->readListBegin(_etype882, _size879);
-            (*(this->success)).resize(_size879);
-            uint32_t _i883;
-            for (_i883 = 0; _i883 < _size879; ++_i883)
+            uint32_t _size887;
+            ::apache::thrift::protocol::TType _etype890;
+            xfer += iprot->readListBegin(_etype890, _size887);
+            (*(this->success)).resize(_size887);
+            uint32_t _i891;
+            for (_i891 = 0; _i891 < _size887; ++_i891)
             {
-              xfer += (*(this->success))[_i883].read(iprot);
+              xfer += (*(this->success))[_i891].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13026,14 +13026,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size884;
-            ::apache::thrift::protocol::TType _etype887;
-            xfer += iprot->readListBegin(_etype887, _size884);
-            this->group_names.resize(_size884);
-            uint32_t _i888;
-            for (_i888 = 0; _i888 < _size884; ++_i888)
+            uint32_t _size892;
+            ::apache::thrift::protocol::TType _etype895;
+            xfer += iprot->readListBegin(_etype895, _size892);
+            this->group_names.resize(_size892);
+            uint32_t _i896;
+            for (_i896 = 0; _i896 < _size892; ++_i896)
             {
-              xfer += iprot->readString(this->group_names[_i888]);
+              xfer += iprot->readString(this->group_names[_i896]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13078,10 +13078,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter889;
-    for (_iter889 = this->group_names.begin(); _iter889 != this->group_names.end(); ++_iter889)
+    std::vector<std::string> ::const_iterator _iter897;
+    for (_iter897 = this->group_names.begin(); _iter897 != this->group_names.end(); ++_iter897)
     {
-      xfer += oprot->writeString((*_iter889));
+      xfer += oprot->writeString((*_iter897));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13121,10 +13121,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter890;
-    for (_iter890 = (*(this->group_names)).begin(); _iter890 != (*(this->group_names)).end(); ++_iter890)
+    std::vector<std::string> ::const_iterator _iter898;
+    for (_iter898 = (*(this->group_names)).begin(); _iter898 != (*(this->group_names)).end(); ++_iter898)
     {
-      xfer += oprot->writeString((*_iter890));
+      xfer += oprot->writeString((*_iter898));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13165,14 +13165,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size891;
-            ::apache::thrift::protocol::TType _etype894;
-            xfer += iprot->readListBegin(_etype894, _size891);
-            this->success.resize(_size891);
-            uint32_t _i895;
-            for (_i895 = 0; _i895 < _size891; ++_i895)
+            uint32_t _size899;
+            ::apache::thrift::protocol::TType _etype902;
+            xfer += iprot->readListBegin(_etype902, _size899);
+            this->success.resize(_size899);
+            uint32_t _i903;
+            for (_i903 = 0; _i903 < _size899; ++_i903)
             {
-              xfer += this->success[_i895].read(iprot);
+              xfer += this->success[_i903].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13219,10 +13219,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter896;
-      for (_iter896 = this->success.begin(); _iter896 != this->success.end(); ++_iter896)
+      std::vector<Partition> ::const_iterator _iter904;
+      for (_iter904 = this->success.begin(); _iter904 != this->success.end(); ++_iter904)
       {
-        xfer += (*_iter896).write(oprot);
+        xfer += (*_iter904).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13271,14 +13271,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size897;
-            ::apache::thrift::protocol::TType _etype900;
-            xfer += iprot->readListBegin(_etype900, _size897);
-            (*(this->success)).resize(_size897);
-            uint32_t _i901;
-            for (_i901 = 0; _i901 < _size897; ++_i901)
+            uint32_t _size905;
+            ::apache::thrift::protocol::TType _etype908;
+            xfer += iprot->readListBegin(_etype908, _size905);
+            (*(this->success)).resize(_size905);
+            uint32_t _i909;
+            for (_i909 = 0; _i909 < _size905; ++_i909)
             {
-              xfer += (*(this->success))[_i901].read(iprot);
+              xfer += (*(this->success))[_i909].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13456,14 +13456,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size902;
-            ::apache::thrift::protocol::TType _etype905;
-            xfer += iprot->readListBegin(_etype905, _size902);
-            this->success.resize(_size902);
-            uint32_t _i906;
-            for (_i906 = 0; _i906 < _size902; ++_i906)
+            uint32_t _size910;
+            ::apache::thrift::protocol::TType _etype913;
+            xfer += iprot->readListBegin(_etype913, _size910);
+            this->success.resize(_size910);
+            uint32_t _i914;
+            for (_i914 = 0; _i914 < _size910; ++_i914)
             {
-              xfer += this->success[_i906].read(iprot);
+              xfer += this->success[_i914].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13510,10 +13510,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter907;
-      for (_iter907 = this->success.begin(); _iter907 != this->success.end(); ++_iter907)
+      std::vector<PartitionSpec> ::const_iterator _iter915;
+      for (_iter915 = this->success.begin(); _iter915 != this->success.end(); ++_iter915)
       {
-        xfer += (*_iter907).write(oprot);
+        xfer += (*_iter915).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13562,14 +13562,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size908;
-            ::apache::thrift::protocol::TType _etype911;
-            xfer += iprot->readListBegin(_etype911, _size908);
-            (*(this->success)).resize(_size908);
-            uint32_t _i912;
-            for (_i912 = 0; _i912 < _size908; ++_i912)
+            uint32_t _size916;
+            ::apache::thrift::protocol::TType _etype919;
+            xfer += iprot->readListBegin(_etype919, _size916);
+            (*(this->success)).resize(_size916);
+            uint32_t _i920;
+            for (_i920 = 0; _i920 < _size916; ++_i920)
             {
-              xfer += (*(this->success))[_i912].read(iprot);
+              xfer += (*(this->success))[_i920].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13747,14 +13747,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _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 _size921;
+            ::apache::thrift::protocol::TType _etype924;
+            xfer += iprot->readListBegin(_etype924, _size921);
+            this->success.resize(_size921);
+            uint32_t _i925;
+            for (_i925 = 0; _i925 < _size921; ++_i925)
             {
-              xfer += iprot->readString(this->success[_i917]);
+              xfer += iprot->readString(this->success[_i925]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13793,10 +13793,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter918;
-      for (_iter918 = this->success.begin(); _iter918 != this->success.end(); ++_iter918)
+      std::vector<std::string> ::const_iterator _iter926;
+      for (_iter926 = this->success.begin(); _iter926 != this->success.end(); ++_iter926)
       {
-        xfer += oprot->writeString((*_iter918));
+        xfer += oprot->writeString((*_iter926));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13841,14 +13841,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size919;
-            ::apache::thrift::protocol::TType _etype922;
-            xfer += iprot->readListBegin(_etype922, _size919);
-            (*(this->success)).resize(_size919);
-            uint32_t _i923;
-            for (_i923 = 0; _i923 < _size919; ++_i923)
+            uint32_t _size927;
+            ::apache::thrift::protocol::TType _etype930;
+            xfer += iprot->readListBegin(_etype930, _size927);
+            (*(this->success)).resize(_size927);
+            uint32_t _i931;
+            for (_i931 = 0; _i931 < _size927; ++_i931)
             {
-              xfer += iprot->readString((*(this->success))[_i923]);
+              xfer += iprot->readString((*(this->success))[_i931]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13923,14 +13923,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _etype927;
-            xfer += iprot->readListBegin(_etype927, _size924);
-            this->part_vals.resize(_size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size932;
+            ::apache::thrift::protocol::TType _etype935;
+            xfer += iprot->readListBegin(_etype935, _size932);
+            this->part_vals.resize(_size932);
+            uint32_t _i936;
+            for (_i936 = 0; _i936 < _size932; ++_i936)
             {
-              xfer += iprot->readString(this->part_vals[_i928]);
+              xfer += iprot->readString(this->part_vals[_i936]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13975,10 +13975,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter929;
-    for (_iter929 = this->part_vals.begin(); _iter929 != this->part_vals.end(); ++_iter929)
+    std::vector<std::string> ::const_iterator _iter937;
+    for (_iter937 = this->part_vals.begin(); _iter937 != this->part_vals.end(); ++_iter937)
     {
-      xfer += oprot->writeString((*_iter929));
+      xfer += oprot->writeString((*_iter937));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14014,10 +14014,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter930;
-    for (_iter930 = (*(this->part_vals)).begin(); _iter930 != (*(this->part_vals)).end(); ++_iter930)
+    std::vector<std::string> ::const_iterator _iter938;
+    for (_iter938 = (*(this->part_vals)).begin(); _iter938 != (*(this->part_vals)).end(); ++_iter938)
     {
-      xfer += oprot->writeString((*_iter930));
+      xfer += oprot->writeString((*_iter938));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14062,14 +14062,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size931;
-            ::apache::thrift::protocol::TType _etype934;
-            xfer += iprot->readListBegin(_etype934, _size931);
-            this->success.resize(_size931);
-            uint32_t _i935;
-            for (_i935 = 0; _i935 < _size931; ++_i935)
+            uint32_t _size939;
+            ::apache::thrift::protocol::TType _etype942;
+            xfer += iprot->readListBegin(_etype942, _size939);
+            this->success.resize(_size939);
+            uint32_t _i943;
+            for (_i943 = 0; _i943 < _size939; ++_i943)
             {
-              xfer += this->success[_i935].read(iprot);
+              xfer += this->success[_i943].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14116,10 +14116,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter936;
-      for (_iter936 = this->success.begin(); _iter936 != this->success.end(); ++_iter936)
+      std::vector<Partition> ::const_iterator _iter944;
+      for (_iter944 = this->success.begin(); _iter944 != this->success.end(); ++_iter944)
       {
-        xfer += (*_iter936).write(oprot);
+        xfer += (*_iter944).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14168,14 +14168,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size937;
-            ::apache::thrift::protocol::TType _etype940;
-            xfer += iprot->readListBegin(_etype940, _size937);
-            (*(this->success)).resize(_size937);
-            uint32_t _i941;
-            for (_i941 = 0; _i941 < _size937; ++_i941)
+            uint32_t _size945;
+            ::apache::thrift::protocol::TType _etype948;
+            xfer += iprot->readListBegin(_etype948, _size945);
+            (*(this->success)).resize(_size945);
+            uint32_t _i949;
+            for (_i949 = 0; _i949 < _size945; ++_i949)
             {
-              xfer += (*(this->success))[_i941].read(iprot);
+              xfer += (*(this->success))[_i949].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14258,14 +14258,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size942;
-            ::apache::thrift::protocol::TType _etype945;
-            xfer += iprot->readListBegin(_etype945, _size942);
-            this->part_vals.resize(_size942);
-            uint32_t _i946;
-            for (_i946 = 0; _i946 < _size942; ++_i946)
+            uint32_t _size950;
+            ::apache::thrift::protocol::TType _etype953;
+            xfer += iprot->readListBegin(_etype953, _size950);
+            this->part_vals.resize(_size950);
+            uint32_t _i954;
+            for (_i954 = 0; _i954 < _size950; ++_i954)
             {
-              xfer += iprot->readString(this->part_vals[_i946]);
+              xfer += iprot->readString(this->part_vals[_i954]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14294,14 +14294,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size947;
-            ::apache::thrift::protocol::TType _etype950;
-            xfer += iprot->readListBegin(_etype950, _size947);
-            this->group_names.resize(_size947);
-            uint32_t _i951;
-            for (_i951 = 0; _i951 < _size947; ++_i951)
+            uint32_t _size955;
+            ::apache::thrift::protocol::TType _etype958;
+            xfer += iprot->readListBegin(_etype958, _size955);
+            this->group_names.resize(_size955);
+            uint32_t _i959;
+            for (_i959 = 0; _i959 < _size955; ++_i959)
             {
-              xfer += iprot->readString(this->group_names[_i951]);
+              xfer += iprot->readString(this->group_names[_i959]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14338,10 +14338,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter952;
-    for (_iter952 = this->part_vals.begin(); _iter952 != this->part_vals.end(); ++_iter952)
+    std::vector<std::string> ::const_iterator _iter960;
+    for (_iter960 = this->part_vals.begin(); _iter960 != this->part_vals.end(); ++_iter960)
     {
-      xfer += oprot->writeString((*_iter952));
+      xfer += oprot->writeString((*_iter960));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14358,10 +14358,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter953;
-    for (_iter953 = this->group_names.begin(); _iter953 != this->group_names.end(); ++_iter953)
+    std::vector<std::string> ::const_iterator _iter961;
+    for (_iter961 = this->group_names.begin(); _iter961 != this->group_names.end(); ++_iter961)
     {
-      xfer += oprot->writeString((*_iter953));
+      xfer += oprot->writeString((*_iter961));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14393,10 +14393,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs

<TRUNCATED>

[8/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)


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

Branch: refs/heads/branch-1
Commit: aecb0c02eaf7b2ee5e448c3aaa8bda1274de78cf
Parents: f1950fc
Author: Wei Zheng <we...@apache.org>
Authored: Tue May 10 11:05:30 2016 -0700
Committer: Wei Zheng <we...@apache.org>
Committed: Tue May 10 11:05:30 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/JavaUtils.java    |    5 +
 metastore/if/hive_metastore.thrift              |    5 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2195 ++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  125 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 1060 +++---
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   42 +
 .../hive/metastore/api/AbortTxnsRequest.java    |  438 +++
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |   32 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 3588 +++++++++++-------
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1311 ++++---
 .../src/gen/thrift/gen-php/metastore/Types.php  |  307 +-
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  931 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  205 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   17 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   58 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |    6 +
 .../hive/metastore/HiveMetaStoreClient.java     |    6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    6 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   69 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |    9 +
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   13 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    9 +
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   19 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   11 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    2 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../hadoop/hive/ql/plan/AbortTxnsDesc.java      |   36 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |   16 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |    4 +-
 .../authorization/plugin/HiveOperationType.java |    1 +
 .../plugin/sqlstd/Operation2Privilege.java      |    3 +-
 .../queries/clientpositive/dbtxnmgr_abort.q     |    6 +
 .../results/clientpositive/dbtxnmgr_abort.q.out |    8 +
 44 files changed, 6842 insertions(+), 4024 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
index dc3a4ae..5bdf6f4 100644
--- a/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
@@ -26,6 +26,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URLClassLoader;
 import java.util.Arrays;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -151,6 +152,10 @@ public final class JavaUtils {
     return "txnid:" + txnId;
   }
 
+  public static String txnIdsToString(List<Long> txnIds) {
+    return "Transactions requested to be aborted: " + txnIds.toString();
+  }
+
   private JavaUtils() {
     // prevent instantiation
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index f84b2a9..4b5d207 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -573,6 +573,10 @@ struct AbortTxnRequest {
     1: required i64 txnid,
 }
 
+struct AbortTxnsRequest {
+    1: required list<i64> txn_ids,
+}
+
 struct CommitTxnRequest {
     1: required i64 txnid,
 }
@@ -1203,6 +1207,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   GetOpenTxnsInfoResponse get_open_txns_info()
   OpenTxnsResponse open_txns(1:OpenTxnRequest rqst)
   void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1)
+  void abort_txns(1:AbortTxnsRequest rqst) throws (1:NoSuchTxnException o1)
   void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
   LockResponse lock(1:LockRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2)
   LockResponse check_lock(1:CheckLockRequest rqst)


[2/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 4fc2da6..fe25366 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -10850,6 +10850,107 @@ class AbortTxnRequest {
 
 }
 
+class AbortTxnsRequest {
+  static $_TSPEC;
+
+  /**
+   * @var int[]
+   */
+  public $txn_ids = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'txn_ids',
+          'type' => TType::LST,
+          'etype' => TType::I64,
+          'elem' => array(
+            'type' => TType::I64,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['txn_ids'])) {
+        $this->txn_ids = $vals['txn_ids'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'AbortTxnsRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->txn_ids = array();
+            $_size400 = 0;
+            $_etype403 = 0;
+            $xfer += $input->readListBegin($_etype403, $_size400);
+            for ($_i404 = 0; $_i404 < $_size400; ++$_i404)
+            {
+              $elem405 = null;
+              $xfer += $input->readI64($elem405);
+              $this->txn_ids []= $elem405;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('AbortTxnsRequest');
+    if ($this->txn_ids !== null) {
+      if (!is_array($this->txn_ids)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('txn_ids', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::I64, count($this->txn_ids));
+        {
+          foreach ($this->txn_ids as $iter406)
+          {
+            $xfer += $output->writeI64($iter406);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class CommitTxnRequest {
   static $_TSPEC;
 
@@ -11187,15 +11288,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size400 = 0;
-            $_etype403 = 0;
-            $xfer += $input->readListBegin($_etype403, $_size400);
-            for ($_i404 = 0; $_i404 < $_size400; ++$_i404)
+            $_size407 = 0;
+            $_etype410 = 0;
+            $xfer += $input->readListBegin($_etype410, $_size407);
+            for ($_i411 = 0; $_i411 < $_size407; ++$_i411)
             {
-              $elem405 = null;
-              $elem405 = new \metastore\LockComponent();
-              $xfer += $elem405->read($input);
-              $this->component []= $elem405;
+              $elem412 = null;
+              $elem412 = new \metastore\LockComponent();
+              $xfer += $elem412->read($input);
+              $this->component []= $elem412;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11251,9 +11352,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter406)
+          foreach ($this->component as $iter413)
           {
-            $xfer += $iter406->write($output);
+            $xfer += $iter413->write($output);
           }
         }
         $output->writeListEnd();
@@ -12196,15 +12297,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size407 = 0;
-            $_etype410 = 0;
-            $xfer += $input->readListBegin($_etype410, $_size407);
-            for ($_i411 = 0; $_i411 < $_size407; ++$_i411)
+            $_size414 = 0;
+            $_etype417 = 0;
+            $xfer += $input->readListBegin($_etype417, $_size414);
+            for ($_i418 = 0; $_i418 < $_size414; ++$_i418)
             {
-              $elem412 = null;
-              $elem412 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem412->read($input);
-              $this->locks []= $elem412;
+              $elem419 = null;
+              $elem419 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem419->read($input);
+              $this->locks []= $elem419;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12232,9 +12333,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter413)
+          foreach ($this->locks as $iter420)
           {
-            $xfer += $iter413->write($output);
+            $xfer += $iter420->write($output);
           }
         }
         $output->writeListEnd();
@@ -12509,17 +12610,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size414 = 0;
-            $_etype417 = 0;
-            $xfer += $input->readSetBegin($_etype417, $_size414);
-            for ($_i418 = 0; $_i418 < $_size414; ++$_i418)
+            $_size421 = 0;
+            $_etype424 = 0;
+            $xfer += $input->readSetBegin($_etype424, $_size421);
+            for ($_i425 = 0; $_i425 < $_size421; ++$_i425)
             {
-              $elem419 = null;
-              $xfer += $input->readI64($elem419);
-              if (is_scalar($elem419)) {
-                $this->aborted[$elem419] = true;
+              $elem426 = null;
+              $xfer += $input->readI64($elem426);
+              if (is_scalar($elem426)) {
+                $this->aborted[$elem426] = true;
               } else {
-                $this->aborted []= $elem419;
+                $this->aborted []= $elem426;
               }
             }
             $xfer += $input->readSetEnd();
@@ -12530,17 +12631,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size420 = 0;
-            $_etype423 = 0;
-            $xfer += $input->readSetBegin($_etype423, $_size420);
-            for ($_i424 = 0; $_i424 < $_size420; ++$_i424)
+            $_size427 = 0;
+            $_etype430 = 0;
+            $xfer += $input->readSetBegin($_etype430, $_size427);
+            for ($_i431 = 0; $_i431 < $_size427; ++$_i431)
             {
-              $elem425 = null;
-              $xfer += $input->readI64($elem425);
-              if (is_scalar($elem425)) {
-                $this->nosuch[$elem425] = true;
+              $elem432 = null;
+              $xfer += $input->readI64($elem432);
+              if (is_scalar($elem432)) {
+                $this->nosuch[$elem432] = true;
               } else {
-                $this->nosuch []= $elem425;
+                $this->nosuch []= $elem432;
               }
             }
             $xfer += $input->readSetEnd();
@@ -12569,12 +12670,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter426 => $iter427)
+          foreach ($this->aborted as $iter433 => $iter434)
           {
-            if (is_scalar($iter427)) {
-            $xfer += $output->writeI64($iter426);
+            if (is_scalar($iter434)) {
+            $xfer += $output->writeI64($iter433);
             } else {
-            $xfer += $output->writeI64($iter427);
+            $xfer += $output->writeI64($iter434);
             }
           }
         }
@@ -12590,12 +12691,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter428 => $iter429)
+          foreach ($this->nosuch as $iter435 => $iter436)
           {
-            if (is_scalar($iter429)) {
-            $xfer += $output->writeI64($iter428);
+            if (is_scalar($iter436)) {
+            $xfer += $output->writeI64($iter435);
             } else {
-            $xfer += $output->writeI64($iter429);
+            $xfer += $output->writeI64($iter436);
             }
           }
         }
@@ -13206,15 +13307,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size430 = 0;
-            $_etype433 = 0;
-            $xfer += $input->readListBegin($_etype433, $_size430);
-            for ($_i434 = 0; $_i434 < $_size430; ++$_i434)
+            $_size437 = 0;
+            $_etype440 = 0;
+            $xfer += $input->readListBegin($_etype440, $_size437);
+            for ($_i441 = 0; $_i441 < $_size437; ++$_i441)
             {
-              $elem435 = null;
-              $elem435 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem435->read($input);
-              $this->compacts []= $elem435;
+              $elem442 = null;
+              $elem442 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem442->read($input);
+              $this->compacts []= $elem442;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13242,9 +13343,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter436)
+          foreach ($this->compacts as $iter443)
           {
-            $xfer += $iter436->write($output);
+            $xfer += $iter443->write($output);
           }
         }
         $output->writeListEnd();
@@ -13362,14 +13463,14 @@ class AddDynamicPartitions {
         case 4:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size437 = 0;
-            $_etype440 = 0;
-            $xfer += $input->readListBegin($_etype440, $_size437);
-            for ($_i441 = 0; $_i441 < $_size437; ++$_i441)
+            $_size444 = 0;
+            $_etype447 = 0;
+            $xfer += $input->readListBegin($_etype447, $_size444);
+            for ($_i448 = 0; $_i448 < $_size444; ++$_i448)
             {
-              $elem442 = null;
-              $xfer += $input->readString($elem442);
-              $this->partitionnames []= $elem442;
+              $elem449 = null;
+              $xfer += $input->readString($elem449);
+              $this->partitionnames []= $elem449;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13412,9 +13513,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter443)
+          foreach ($this->partitionnames as $iter450)
           {
-            $xfer += $output->writeString($iter443);
+            $xfer += $output->writeString($iter450);
           }
         }
         $output->writeListEnd();
@@ -13767,15 +13868,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size444 = 0;
-            $_etype447 = 0;
-            $xfer += $input->readListBegin($_etype447, $_size444);
-            for ($_i448 = 0; $_i448 < $_size444; ++$_i448)
+            $_size451 = 0;
+            $_etype454 = 0;
+            $xfer += $input->readListBegin($_etype454, $_size451);
+            for ($_i455 = 0; $_i455 < $_size451; ++$_i455)
             {
-              $elem449 = null;
-              $elem449 = new \metastore\NotificationEvent();
-              $xfer += $elem449->read($input);
-              $this->events []= $elem449;
+              $elem456 = null;
+              $elem456 = new \metastore\NotificationEvent();
+              $xfer += $elem456->read($input);
+              $this->events []= $elem456;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13803,9 +13904,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter450)
+          foreach ($this->events as $iter457)
           {
-            $xfer += $iter450->write($output);
+            $xfer += $iter457->write($output);
           }
         }
         $output->writeListEnd();
@@ -13944,14 +14045,14 @@ class InsertEventRequestData {
         case 1:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size451 = 0;
-            $_etype454 = 0;
-            $xfer += $input->readListBegin($_etype454, $_size451);
-            for ($_i455 = 0; $_i455 < $_size451; ++$_i455)
+            $_size458 = 0;
+            $_etype461 = 0;
+            $xfer += $input->readListBegin($_etype461, $_size458);
+            for ($_i462 = 0; $_i462 < $_size458; ++$_i462)
             {
-              $elem456 = null;
-              $xfer += $input->readString($elem456);
-              $this->filesAdded []= $elem456;
+              $elem463 = null;
+              $xfer += $input->readString($elem463);
+              $this->filesAdded []= $elem463;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13979,9 +14080,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter457)
+          foreach ($this->filesAdded as $iter464)
           {
-            $xfer += $output->writeString($iter457);
+            $xfer += $output->writeString($iter464);
           }
         }
         $output->writeListEnd();
@@ -14199,14 +14300,14 @@ class FireEventRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionVals = array();
-            $_size458 = 0;
-            $_etype461 = 0;
-            $xfer += $input->readListBegin($_etype461, $_size458);
-            for ($_i462 = 0; $_i462 < $_size458; ++$_i462)
+            $_size465 = 0;
+            $_etype468 = 0;
+            $xfer += $input->readListBegin($_etype468, $_size465);
+            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
             {
-              $elem463 = null;
-              $xfer += $input->readString($elem463);
-              $this->partitionVals []= $elem463;
+              $elem470 = null;
+              $xfer += $input->readString($elem470);
+              $this->partitionVals []= $elem470;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14257,9 +14358,9 @@ class FireEventRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter464)
+          foreach ($this->partitionVals as $iter471)
           {
-            $xfer += $output->writeString($iter464);
+            $xfer += $output->writeString($iter471);
           }
         }
         $output->writeListEnd();
@@ -14374,15 +14475,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size465 = 0;
-            $_etype468 = 0;
-            $xfer += $input->readListBegin($_etype468, $_size465);
-            for ($_i469 = 0; $_i469 < $_size465; ++$_i469)
+            $_size472 = 0;
+            $_etype475 = 0;
+            $xfer += $input->readListBegin($_etype475, $_size472);
+            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
             {
-              $elem470 = null;
-              $elem470 = new \metastore\Function();
-              $xfer += $elem470->read($input);
-              $this->functions []= $elem470;
+              $elem477 = null;
+              $elem477 = new \metastore\Function();
+              $xfer += $elem477->read($input);
+              $this->functions []= $elem477;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14410,9 +14511,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter471)
+          foreach ($this->functions as $iter478)
           {
-            $xfer += $iter471->write($output);
+            $xfer += $iter478->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index b3182a8..36c4e3e 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -136,6 +136,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetOpenTxnsInfoResponse get_open_txns_info()')
   print('  OpenTxnsResponse open_txns(OpenTxnRequest rqst)')
   print('  void abort_txn(AbortTxnRequest rqst)')
+  print('  void abort_txns(AbortTxnsRequest rqst)')
   print('  void commit_txn(CommitTxnRequest rqst)')
   print('  LockResponse lock(LockRequest rqst)')
   print('  LockResponse check_lock(CheckLockRequest rqst)')
@@ -890,6 +891,12 @@ elif cmd == 'abort_txn':
     sys.exit(1)
   pp.pprint(client.abort_txn(eval(args[0]),))
 
+elif cmd == 'abort_txns':
+  if len(args) != 1:
+    print('abort_txns requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.abort_txns(eval(args[0]),))
+
 elif cmd == 'commit_txn':
   if len(args) != 1:
     print('commit_txn requires 1 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 1cc6acf..d739752 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -946,6 +946,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def abort_txns(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
   def commit_txn(self, rqst):
     """
     Parameters:
@@ -5232,6 +5239,37 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     return
 
+  def abort_txns(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_abort_txns(rqst)
+    self.recv_abort_txns()
+
+  def send_abort_txns(self, rqst):
+    self._oprot.writeMessageBegin('abort_txns', TMessageType.CALL, self._seqid)
+    args = abort_txns_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_abort_txns(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = abort_txns_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    return
+
   def commit_txn(self, rqst):
     """
     Parameters:
@@ -5764,6 +5802,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_open_txns_info"] = Processor.process_get_open_txns_info
     self._processMap["open_txns"] = Processor.process_open_txns
     self._processMap["abort_txn"] = Processor.process_abort_txn
+    self._processMap["abort_txns"] = Processor.process_abort_txns
     self._processMap["commit_txn"] = Processor.process_commit_txn
     self._processMap["lock"] = Processor.process_lock
     self._processMap["check_lock"] = Processor.process_check_lock
@@ -8626,6 +8665,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_abort_txns(self, seqid, iprot, oprot):
+    args = abort_txns_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = abort_txns_result()
+    try:
+      self._handler.abort_txns(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchTxnException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("abort_txns", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_commit_txn(self, seqid, iprot, oprot):
     args = commit_txn_args()
     args.read(iprot)
@@ -9801,10 +9862,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype472, _size469) = iprot.readListBegin()
-          for _i473 in xrange(_size469):
-            _elem474 = iprot.readString()
-            self.success.append(_elem474)
+          (_etype479, _size476) = iprot.readListBegin()
+          for _i480 in xrange(_size476):
+            _elem481 = iprot.readString()
+            self.success.append(_elem481)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9827,8 +9888,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter475 in self.success:
-        oprot.writeString(iter475)
+      for iter482 in self.success:
+        oprot.writeString(iter482)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -9933,10 +9994,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype479, _size476) = iprot.readListBegin()
-          for _i480 in xrange(_size476):
-            _elem481 = iprot.readString()
-            self.success.append(_elem481)
+          (_etype486, _size483) = iprot.readListBegin()
+          for _i487 in xrange(_size483):
+            _elem488 = iprot.readString()
+            self.success.append(_elem488)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -9959,8 +10020,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter482 in self.success:
-        oprot.writeString(iter482)
+      for iter489 in self.success:
+        oprot.writeString(iter489)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -10730,12 +10791,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype484, _vtype485, _size483 ) = iprot.readMapBegin()
-          for _i487 in xrange(_size483):
-            _key488 = iprot.readString()
-            _val489 = Type()
-            _val489.read(iprot)
-            self.success[_key488] = _val489
+          (_ktype491, _vtype492, _size490 ) = iprot.readMapBegin()
+          for _i494 in xrange(_size490):
+            _key495 = iprot.readString()
+            _val496 = Type()
+            _val496.read(iprot)
+            self.success[_key495] = _val496
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -10758,9 +10819,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter490,viter491 in self.success.items():
-        oprot.writeString(kiter490)
-        viter491.write(oprot)
+      for kiter497,viter498 in self.success.items():
+        oprot.writeString(kiter497)
+        viter498.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -10903,11 +10964,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype495, _size492) = iprot.readListBegin()
-          for _i496 in xrange(_size492):
-            _elem497 = FieldSchema()
-            _elem497.read(iprot)
-            self.success.append(_elem497)
+          (_etype502, _size499) = iprot.readListBegin()
+          for _i503 in xrange(_size499):
+            _elem504 = FieldSchema()
+            _elem504.read(iprot)
+            self.success.append(_elem504)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -10942,8 +11003,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter498 in self.success:
-        iter498.write(oprot)
+      for iter505 in self.success:
+        iter505.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11110,11 +11171,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype502, _size499) = iprot.readListBegin()
-          for _i503 in xrange(_size499):
-            _elem504 = FieldSchema()
-            _elem504.read(iprot)
-            self.success.append(_elem504)
+          (_etype509, _size506) = iprot.readListBegin()
+          for _i510 in xrange(_size506):
+            _elem511 = FieldSchema()
+            _elem511.read(iprot)
+            self.success.append(_elem511)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11149,8 +11210,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter505 in self.success:
-        iter505.write(oprot)
+      for iter512 in self.success:
+        iter512.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11303,11 +11364,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype509, _size506) = iprot.readListBegin()
-          for _i510 in xrange(_size506):
-            _elem511 = FieldSchema()
-            _elem511.read(iprot)
-            self.success.append(_elem511)
+          (_etype516, _size513) = iprot.readListBegin()
+          for _i517 in xrange(_size513):
+            _elem518 = FieldSchema()
+            _elem518.read(iprot)
+            self.success.append(_elem518)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11342,8 +11403,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter512 in self.success:
-        iter512.write(oprot)
+      for iter519 in self.success:
+        iter519.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -11510,11 +11571,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype516, _size513) = iprot.readListBegin()
-          for _i517 in xrange(_size513):
-            _elem518 = FieldSchema()
-            _elem518.read(iprot)
-            self.success.append(_elem518)
+          (_etype523, _size520) = iprot.readListBegin()
+          for _i524 in xrange(_size520):
+            _elem525 = FieldSchema()
+            _elem525.read(iprot)
+            self.success.append(_elem525)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11549,8 +11610,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter519 in self.success:
-        iter519.write(oprot)
+      for iter526 in self.success:
+        iter526.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12415,10 +12476,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype523, _size520) = iprot.readListBegin()
-          for _i524 in xrange(_size520):
-            _elem525 = iprot.readString()
-            self.success.append(_elem525)
+          (_etype530, _size527) = iprot.readListBegin()
+          for _i531 in xrange(_size527):
+            _elem532 = iprot.readString()
+            self.success.append(_elem532)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12441,8 +12502,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter526 in self.success:
-        oprot.writeString(iter526)
+      for iter533 in self.success:
+        oprot.writeString(iter533)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12566,10 +12627,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype530, _size527) = iprot.readListBegin()
-          for _i531 in xrange(_size527):
-            _elem532 = iprot.readString()
-            self.success.append(_elem532)
+          (_etype537, _size534) = iprot.readListBegin()
+          for _i538 in xrange(_size534):
+            _elem539 = iprot.readString()
+            self.success.append(_elem539)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12592,8 +12653,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter533 in self.success:
-        oprot.writeString(iter533)
+      for iter540 in self.success:
+        oprot.writeString(iter540)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -12829,10 +12890,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype537, _size534) = iprot.readListBegin()
-          for _i538 in xrange(_size534):
-            _elem539 = iprot.readString()
-            self.tbl_names.append(_elem539)
+          (_etype544, _size541) = iprot.readListBegin()
+          for _i545 in xrange(_size541):
+            _elem546 = iprot.readString()
+            self.tbl_names.append(_elem546)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12853,8 +12914,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter540 in self.tbl_names:
-        oprot.writeString(iter540)
+      for iter547 in self.tbl_names:
+        oprot.writeString(iter547)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12915,11 +12976,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype544, _size541) = iprot.readListBegin()
-          for _i545 in xrange(_size541):
-            _elem546 = Table()
-            _elem546.read(iprot)
-            self.success.append(_elem546)
+          (_etype551, _size548) = iprot.readListBegin()
+          for _i552 in xrange(_size548):
+            _elem553 = Table()
+            _elem553.read(iprot)
+            self.success.append(_elem553)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12954,8 +13015,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter547 in self.success:
-        iter547.write(oprot)
+      for iter554 in self.success:
+        iter554.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13121,10 +13182,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype551, _size548) = iprot.readListBegin()
-          for _i552 in xrange(_size548):
-            _elem553 = iprot.readString()
-            self.success.append(_elem553)
+          (_etype558, _size555) = iprot.readListBegin()
+          for _i559 in xrange(_size555):
+            _elem560 = iprot.readString()
+            self.success.append(_elem560)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13159,8 +13220,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter554 in self.success:
-        oprot.writeString(iter554)
+      for iter561 in self.success:
+        oprot.writeString(iter561)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14130,11 +14191,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype558, _size555) = iprot.readListBegin()
-          for _i559 in xrange(_size555):
-            _elem560 = Partition()
-            _elem560.read(iprot)
-            self.new_parts.append(_elem560)
+          (_etype565, _size562) = iprot.readListBegin()
+          for _i566 in xrange(_size562):
+            _elem567 = Partition()
+            _elem567.read(iprot)
+            self.new_parts.append(_elem567)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14151,8 +14212,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter561 in self.new_parts:
-        iter561.write(oprot)
+      for iter568 in self.new_parts:
+        iter568.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14310,11 +14371,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype565, _size562) = iprot.readListBegin()
-          for _i566 in xrange(_size562):
-            _elem567 = PartitionSpec()
-            _elem567.read(iprot)
-            self.new_parts.append(_elem567)
+          (_etype572, _size569) = iprot.readListBegin()
+          for _i573 in xrange(_size569):
+            _elem574 = PartitionSpec()
+            _elem574.read(iprot)
+            self.new_parts.append(_elem574)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14331,8 +14392,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter568 in self.new_parts:
-        iter568.write(oprot)
+      for iter575 in self.new_parts:
+        iter575.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14506,10 +14567,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype572, _size569) = iprot.readListBegin()
-          for _i573 in xrange(_size569):
-            _elem574 = iprot.readString()
-            self.part_vals.append(_elem574)
+          (_etype579, _size576) = iprot.readListBegin()
+          for _i580 in xrange(_size576):
+            _elem581 = iprot.readString()
+            self.part_vals.append(_elem581)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14534,8 +14595,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter575 in self.part_vals:
-        oprot.writeString(iter575)
+      for iter582 in self.part_vals:
+        oprot.writeString(iter582)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14888,10 +14949,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype579, _size576) = iprot.readListBegin()
-          for _i580 in xrange(_size576):
-            _elem581 = iprot.readString()
-            self.part_vals.append(_elem581)
+          (_etype586, _size583) = iprot.readListBegin()
+          for _i587 in xrange(_size583):
+            _elem588 = iprot.readString()
+            self.part_vals.append(_elem588)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14922,8 +14983,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter582 in self.part_vals:
-        oprot.writeString(iter582)
+      for iter589 in self.part_vals:
+        oprot.writeString(iter589)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -15518,10 +15579,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype586, _size583) = iprot.readListBegin()
-          for _i587 in xrange(_size583):
-            _elem588 = iprot.readString()
-            self.part_vals.append(_elem588)
+          (_etype593, _size590) = iprot.readListBegin()
+          for _i594 in xrange(_size590):
+            _elem595 = iprot.readString()
+            self.part_vals.append(_elem595)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15551,8 +15612,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter589 in self.part_vals:
-        oprot.writeString(iter589)
+      for iter596 in self.part_vals:
+        oprot.writeString(iter596)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -15725,10 +15786,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype593, _size590) = iprot.readListBegin()
-          for _i594 in xrange(_size590):
-            _elem595 = iprot.readString()
-            self.part_vals.append(_elem595)
+          (_etype600, _size597) = iprot.readListBegin()
+          for _i601 in xrange(_size597):
+            _elem602 = iprot.readString()
+            self.part_vals.append(_elem602)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15764,8 +15825,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter596 in self.part_vals:
-        oprot.writeString(iter596)
+      for iter603 in self.part_vals:
+        oprot.writeString(iter603)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -16502,10 +16563,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype600, _size597) = iprot.readListBegin()
-          for _i601 in xrange(_size597):
-            _elem602 = iprot.readString()
-            self.part_vals.append(_elem602)
+          (_etype607, _size604) = iprot.readListBegin()
+          for _i608 in xrange(_size604):
+            _elem609 = iprot.readString()
+            self.part_vals.append(_elem609)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16530,8 +16591,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter603 in self.part_vals:
-        oprot.writeString(iter603)
+      for iter610 in self.part_vals:
+        oprot.writeString(iter610)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16690,11 +16751,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype605, _vtype606, _size604 ) = iprot.readMapBegin()
-          for _i608 in xrange(_size604):
-            _key609 = iprot.readString()
-            _val610 = iprot.readString()
-            self.partitionSpecs[_key609] = _val610
+          (_ktype612, _vtype613, _size611 ) = iprot.readMapBegin()
+          for _i615 in xrange(_size611):
+            _key616 = iprot.readString()
+            _val617 = iprot.readString()
+            self.partitionSpecs[_key616] = _val617
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16731,9 +16792,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter611,viter612 in self.partitionSpecs.items():
-        oprot.writeString(kiter611)
-        oprot.writeString(viter612)
+      for kiter618,viter619 in self.partitionSpecs.items():
+        oprot.writeString(kiter618)
+        oprot.writeString(viter619)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -16938,11 +16999,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype614, _vtype615, _size613 ) = iprot.readMapBegin()
-          for _i617 in xrange(_size613):
-            _key618 = iprot.readString()
-            _val619 = iprot.readString()
-            self.partitionSpecs[_key618] = _val619
+          (_ktype621, _vtype622, _size620 ) = iprot.readMapBegin()
+          for _i624 in xrange(_size620):
+            _key625 = iprot.readString()
+            _val626 = iprot.readString()
+            self.partitionSpecs[_key625] = _val626
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16979,9 +17040,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter620,viter621 in self.partitionSpecs.items():
-        oprot.writeString(kiter620)
-        oprot.writeString(viter621)
+      for kiter627,viter628 in self.partitionSpecs.items():
+        oprot.writeString(kiter627)
+        oprot.writeString(viter628)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -17064,11 +17125,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype625, _size622) = iprot.readListBegin()
-          for _i626 in xrange(_size622):
-            _elem627 = Partition()
-            _elem627.read(iprot)
-            self.success.append(_elem627)
+          (_etype632, _size629) = iprot.readListBegin()
+          for _i633 in xrange(_size629):
+            _elem634 = Partition()
+            _elem634.read(iprot)
+            self.success.append(_elem634)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17109,8 +17170,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter628 in self.success:
-        iter628.write(oprot)
+      for iter635 in self.success:
+        iter635.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17204,10 +17265,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype632, _size629) = iprot.readListBegin()
-          for _i633 in xrange(_size629):
-            _elem634 = iprot.readString()
-            self.part_vals.append(_elem634)
+          (_etype639, _size636) = iprot.readListBegin()
+          for _i640 in xrange(_size636):
+            _elem641 = iprot.readString()
+            self.part_vals.append(_elem641)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17219,10 +17280,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype638, _size635) = iprot.readListBegin()
-          for _i639 in xrange(_size635):
-            _elem640 = iprot.readString()
-            self.group_names.append(_elem640)
+          (_etype645, _size642) = iprot.readListBegin()
+          for _i646 in xrange(_size642):
+            _elem647 = iprot.readString()
+            self.group_names.append(_elem647)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17247,8 +17308,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter641 in self.part_vals:
-        oprot.writeString(iter641)
+      for iter648 in self.part_vals:
+        oprot.writeString(iter648)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -17258,8 +17319,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter642 in self.group_names:
-        oprot.writeString(iter642)
+      for iter649 in self.group_names:
+        oprot.writeString(iter649)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17688,11 +17749,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = Partition()
-            _elem648.read(iprot)
-            self.success.append(_elem648)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = Partition()
+            _elem655.read(iprot)
+            self.success.append(_elem655)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17721,8 +17782,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter649 in self.success:
-        iter649.write(oprot)
+      for iter656 in self.success:
+        iter656.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17816,10 +17877,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype653, _size650) = iprot.readListBegin()
-          for _i654 in xrange(_size650):
-            _elem655 = iprot.readString()
-            self.group_names.append(_elem655)
+          (_etype660, _size657) = iprot.readListBegin()
+          for _i661 in xrange(_size657):
+            _elem662 = iprot.readString()
+            self.group_names.append(_elem662)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17852,8 +17913,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter656 in self.group_names:
-        oprot.writeString(iter656)
+      for iter663 in self.group_names:
+        oprot.writeString(iter663)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17914,11 +17975,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype660, _size657) = iprot.readListBegin()
-          for _i661 in xrange(_size657):
-            _elem662 = Partition()
-            _elem662.read(iprot)
-            self.success.append(_elem662)
+          (_etype667, _size664) = iprot.readListBegin()
+          for _i668 in xrange(_size664):
+            _elem669 = Partition()
+            _elem669.read(iprot)
+            self.success.append(_elem669)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17947,8 +18008,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter663 in self.success:
-        iter663.write(oprot)
+      for iter670 in self.success:
+        iter670.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18106,11 +18167,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype667, _size664) = iprot.readListBegin()
-          for _i668 in xrange(_size664):
-            _elem669 = PartitionSpec()
-            _elem669.read(iprot)
-            self.success.append(_elem669)
+          (_etype674, _size671) = iprot.readListBegin()
+          for _i675 in xrange(_size671):
+            _elem676 = PartitionSpec()
+            _elem676.read(iprot)
+            self.success.append(_elem676)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18139,8 +18200,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter670 in self.success:
-        iter670.write(oprot)
+      for iter677 in self.success:
+        iter677.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18295,10 +18356,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype674, _size671) = iprot.readListBegin()
-          for _i675 in xrange(_size671):
-            _elem676 = iprot.readString()
-            self.success.append(_elem676)
+          (_etype681, _size678) = iprot.readListBegin()
+          for _i682 in xrange(_size678):
+            _elem683 = iprot.readString()
+            self.success.append(_elem683)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18321,8 +18382,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter677 in self.success:
-        oprot.writeString(iter677)
+      for iter684 in self.success:
+        oprot.writeString(iter684)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -18398,10 +18459,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype681, _size678) = iprot.readListBegin()
-          for _i682 in xrange(_size678):
-            _elem683 = iprot.readString()
-            self.part_vals.append(_elem683)
+          (_etype688, _size685) = iprot.readListBegin()
+          for _i689 in xrange(_size685):
+            _elem690 = iprot.readString()
+            self.part_vals.append(_elem690)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18431,8 +18492,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter684 in self.part_vals:
-        oprot.writeString(iter684)
+      for iter691 in self.part_vals:
+        oprot.writeString(iter691)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18496,11 +18557,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype688, _size685) = iprot.readListBegin()
-          for _i689 in xrange(_size685):
-            _elem690 = Partition()
-            _elem690.read(iprot)
-            self.success.append(_elem690)
+          (_etype695, _size692) = iprot.readListBegin()
+          for _i696 in xrange(_size692):
+            _elem697 = Partition()
+            _elem697.read(iprot)
+            self.success.append(_elem697)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18529,8 +18590,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter691 in self.success:
-        iter691.write(oprot)
+      for iter698 in self.success:
+        iter698.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18617,10 +18678,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype695, _size692) = iprot.readListBegin()
-          for _i696 in xrange(_size692):
-            _elem697 = iprot.readString()
-            self.part_vals.append(_elem697)
+          (_etype702, _size699) = iprot.readListBegin()
+          for _i703 in xrange(_size699):
+            _elem704 = iprot.readString()
+            self.part_vals.append(_elem704)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18637,10 +18698,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype701, _size698) = iprot.readListBegin()
-          for _i702 in xrange(_size698):
-            _elem703 = iprot.readString()
-            self.group_names.append(_elem703)
+          (_etype708, _size705) = iprot.readListBegin()
+          for _i709 in xrange(_size705):
+            _elem710 = iprot.readString()
+            self.group_names.append(_elem710)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18665,8 +18726,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter704 in self.part_vals:
-        oprot.writeString(iter704)
+      for iter711 in self.part_vals:
+        oprot.writeString(iter711)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18680,8 +18741,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter705 in self.group_names:
-        oprot.writeString(iter705)
+      for iter712 in self.group_names:
+        oprot.writeString(iter712)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18743,11 +18804,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype709, _size706) = iprot.readListBegin()
-          for _i710 in xrange(_size706):
-            _elem711 = Partition()
-            _elem711.read(iprot)
-            self.success.append(_elem711)
+          (_etype716, _size713) = iprot.readListBegin()
+          for _i717 in xrange(_size713):
+            _elem718 = Partition()
+            _elem718.read(iprot)
+            self.success.append(_elem718)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18776,8 +18837,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter712 in self.success:
-        iter712.write(oprot)
+      for iter719 in self.success:
+        iter719.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18858,10 +18919,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype716, _size713) = iprot.readListBegin()
-          for _i717 in xrange(_size713):
-            _elem718 = iprot.readString()
-            self.part_vals.append(_elem718)
+          (_etype723, _size720) = iprot.readListBegin()
+          for _i724 in xrange(_size720):
+            _elem725 = iprot.readString()
+            self.part_vals.append(_elem725)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18891,8 +18952,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter719 in self.part_vals:
-        oprot.writeString(iter719)
+      for iter726 in self.part_vals:
+        oprot.writeString(iter726)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -18956,10 +19017,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype723, _size720) = iprot.readListBegin()
-          for _i724 in xrange(_size720):
-            _elem725 = iprot.readString()
-            self.success.append(_elem725)
+          (_etype730, _size727) = iprot.readListBegin()
+          for _i731 in xrange(_size727):
+            _elem732 = iprot.readString()
+            self.success.append(_elem732)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18988,8 +19049,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter726 in self.success:
-        oprot.writeString(iter726)
+      for iter733 in self.success:
+        oprot.writeString(iter733)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19160,11 +19221,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype730, _size727) = iprot.readListBegin()
-          for _i731 in xrange(_size727):
-            _elem732 = Partition()
-            _elem732.read(iprot)
-            self.success.append(_elem732)
+          (_etype737, _size734) = iprot.readListBegin()
+          for _i738 in xrange(_size734):
+            _elem739 = Partition()
+            _elem739.read(iprot)
+            self.success.append(_elem739)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19193,8 +19254,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter733 in self.success:
-        iter733.write(oprot)
+      for iter740 in self.success:
+        iter740.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19365,11 +19426,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype737, _size734) = iprot.readListBegin()
-          for _i738 in xrange(_size734):
-            _elem739 = PartitionSpec()
-            _elem739.read(iprot)
-            self.success.append(_elem739)
+          (_etype744, _size741) = iprot.readListBegin()
+          for _i745 in xrange(_size741):
+            _elem746 = PartitionSpec()
+            _elem746.read(iprot)
+            self.success.append(_elem746)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19398,8 +19459,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter740 in self.success:
-        iter740.write(oprot)
+      for iter747 in self.success:
+        iter747.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19636,10 +19697,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype744, _size741) = iprot.readListBegin()
-          for _i745 in xrange(_size741):
-            _elem746 = iprot.readString()
-            self.names.append(_elem746)
+          (_etype751, _size748) = iprot.readListBegin()
+          for _i752 in xrange(_size748):
+            _elem753 = iprot.readString()
+            self.names.append(_elem753)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19664,8 +19725,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter747 in self.names:
-        oprot.writeString(iter747)
+      for iter754 in self.names:
+        oprot.writeString(iter754)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19724,11 +19785,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype751, _size748) = iprot.readListBegin()
-          for _i752 in xrange(_size748):
-            _elem753 = Partition()
-            _elem753.read(iprot)
-            self.success.append(_elem753)
+          (_etype758, _size755) = iprot.readListBegin()
+          for _i759 in xrange(_size755):
+            _elem760 = Partition()
+            _elem760.read(iprot)
+            self.success.append(_elem760)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19757,8 +19818,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter754 in self.success:
-        iter754.write(oprot)
+      for iter761 in self.success:
+        iter761.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20008,11 +20069,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype758, _size755) = iprot.readListBegin()
-          for _i759 in xrange(_size755):
-            _elem760 = Partition()
-            _elem760.read(iprot)
-            self.new_parts.append(_elem760)
+          (_etype765, _size762) = iprot.readListBegin()
+          for _i766 in xrange(_size762):
+            _elem767 = Partition()
+            _elem767.read(iprot)
+            self.new_parts.append(_elem767)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20037,8 +20098,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter761 in self.new_parts:
-        iter761.write(oprot)
+      for iter768 in self.new_parts:
+        iter768.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20377,10 +20438,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype765, _size762) = iprot.readListBegin()
-          for _i766 in xrange(_size762):
-            _elem767 = iprot.readString()
-            self.part_vals.append(_elem767)
+          (_etype772, _size769) = iprot.readListBegin()
+          for _i773 in xrange(_size769):
+            _elem774 = iprot.readString()
+            self.part_vals.append(_elem774)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20411,8 +20472,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter768 in self.part_vals:
-        oprot.writeString(iter768)
+      for iter775 in self.part_vals:
+        oprot.writeString(iter775)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -20554,10 +20615,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype772, _size769) = iprot.readListBegin()
-          for _i773 in xrange(_size769):
-            _elem774 = iprot.readString()
-            self.part_vals.append(_elem774)
+          (_etype779, _size776) = iprot.readListBegin()
+          for _i780 in xrange(_size776):
+            _elem781 = iprot.readString()
+            self.part_vals.append(_elem781)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20579,8 +20640,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter775 in self.part_vals:
-        oprot.writeString(iter775)
+      for iter782 in self.part_vals:
+        oprot.writeString(iter782)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -20938,10 +20999,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype779, _size776) = iprot.readListBegin()
-          for _i780 in xrange(_size776):
-            _elem781 = iprot.readString()
-            self.success.append(_elem781)
+          (_etype786, _size783) = iprot.readListBegin()
+          for _i787 in xrange(_size783):
+            _elem788 = iprot.readString()
+            self.success.append(_elem788)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20964,8 +21025,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter782 in self.success:
-        oprot.writeString(iter782)
+      for iter789 in self.success:
+        oprot.writeString(iter789)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21089,11 +21150,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype784, _vtype785, _size783 ) = iprot.readMapBegin()
-          for _i787 in xrange(_size783):
-            _key788 = iprot.readString()
-            _val789 = iprot.readString()
-            self.success[_key788] = _val789
+          (_ktype791, _vtype792, _size790 ) = iprot.readMapBegin()
+          for _i794 in xrange(_size790):
+            _key795 = iprot.readString()
+            _val796 = iprot.readString()
+            self.success[_key795] = _val796
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21116,9 +21177,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter790,viter791 in self.success.items():
-        oprot.writeString(kiter790)
-        oprot.writeString(viter791)
+      for kiter797,viter798 in self.success.items():
+        oprot.writeString(kiter797)
+        oprot.writeString(viter798)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21194,11 +21255,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype793, _vtype794, _size792 ) = iprot.readMapBegin()
-          for _i796 in xrange(_size792):
-            _key797 = iprot.readString()
-            _val798 = iprot.readString()
-            self.part_vals[_key797] = _val798
+          (_ktype800, _vtype801, _size799 ) = iprot.readMapBegin()
+          for _i803 in xrange(_size799):
+            _key804 = iprot.readString()
+            _val805 = iprot.readString()
+            self.part_vals[_key804] = _val805
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21228,9 +21289,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter799,viter800 in self.part_vals.items():
-        oprot.writeString(kiter799)
-        oprot.writeString(viter800)
+      for kiter806,viter807 in self.part_vals.items():
+        oprot.writeString(kiter806)
+        oprot.writeString(viter807)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -21444,11 +21505,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype802, _vtype803, _size801 ) = iprot.readMapBegin()
-          for _i805 in xrange(_size801):
-            _key806 = iprot.readString()
-            _val807 = iprot.readString()
-            self.part_vals[_key806] = _val807
+          (_ktype809, _vtype810, _size808 ) = iprot.readMapBegin()
+          for _i812 in xrange(_size808):
+            _key813 = iprot.readString()
+            _val814 = iprot.readString()
+            self.part_vals[_key813] = _val814
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21478,9 +21539,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter808,viter809 in self.part_vals.items():
-        oprot.writeString(kiter808)
-        oprot.writeString(viter809)
+      for kiter815,viter816 in self.part_vals.items():
+        oprot.writeString(kiter815)
+        oprot.writeString(viter816)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -22535,11 +22596,11 @@ class get_indexes_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype813, _size810) = iprot.readListBegin()
-          for _i814 in xrange(_size810):
-            _elem815 = Index()
-            _elem815.read(iprot)
-            self.success.append(_elem815)
+          (_etype820, _size817) = iprot.readListBegin()
+          for _i821 in xrange(_size817):
+            _elem822 = Index()
+            _elem822.read(iprot)
+            self.success.append(_elem822)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22568,8 +22629,8 @@ class get_indexes_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for 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:
@@ -22724,10 +22785,10 @@ class get_index_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = iprot.readString()
-            self.success.append(_elem822)
+          (_etype827, _size824) = iprot.readListBegin()
+          for _i828 in xrange(_size824):
+            _elem829 = iprot.readString()
+            self.success.append(_elem829)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22750,8 +22811,8 @@ class get_index_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter823 in self.success:
-        oprot.writeString(iter823)
+      for iter830 in self.success:
+        oprot.writeString(iter830)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -25299,10 +25360,10 @@ class get_functions_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)
@@ -25325,8 +25386,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter830 in self.success:
-        oprot.writeString(iter830)
+      for iter837 in self.success:
+        oprot.writeString(iter837)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26014,10 +26075,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype834, _size831) = iprot.readListBegin()
-          for _i835 in xrange(_size831):
-            _elem836 = iprot.readString()
-            self.success.append(_elem836)
+          (_etype841, _size838) = iprot.readListBegin()
+          for _i842 in xrange(_size838):
+            _elem843 = iprot.readString()
+            self.success.append(_elem843)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26040,8 +26101,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter837 in self.success:
-        oprot.writeString(iter837)
+      for iter844 in self.success:
+        oprot.writeString(iter844)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26555,11 +26616,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype841, _size838) = iprot.readListBegin()
-          for _i842 in xrange(_size838):
-            _elem843 = Role()
-            _elem843.read(iprot)
-            self.success.append(_elem843)
+          (_etype848, _size845) = iprot.readListBegin()
+          for _i849 in xrange(_size845):
+            _elem850 = Role()
+            _elem850.read(iprot)
+            self.success.append(_elem850)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26582,8 +26643,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for 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:
@@ -27092,10 +27153,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype848, _size845) = iprot.readListBegin()
-          for _i849 in xrange(_size845):
-            _elem850 = iprot.readString()
-            self.group_names.append(_elem850)
+          (_etype855, _size852) = iprot.readListBegin()
+          for _i856 in xrange(_size852):
+            _elem857 = iprot.readString()
+            self.group_names.append(_elem857)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27120,8 +27181,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter851 in self.group_names:
-        oprot.writeString(iter851)
+      for iter858 in self.group_names:
+        oprot.writeString(iter858)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27348,11 +27409,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype855, _size852) = iprot.readListBegin()
-          for _i856 in xrange(_size852):
-            _elem857 = HiveObjectPrivilege()
-            _elem857.read(iprot)
-            self.success.append(_elem857)
+          (_etype862, _size859) = iprot.readListBegin()
+          for _i863 in xrange(_size859):
+            _elem864 = HiveObjectPrivilege()
+            _elem864.read(iprot)
+            self.success.append(_elem864)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27375,8 +27436,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter858 in self.success:
-        iter858.write(oprot)
+      for iter865 in self.success:
+        iter865.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27874,10 +27935,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype862, _size859) = iprot.readListBegin()
-          for _i863 in xrange(_size859):
-            _elem864 = iprot.readString()
-            self.group_names.append(_elem864)
+          (_etype869, _size866) = iprot.readListBegin()
+          for _i870 in xrange(_size866):
+            _elem871 = iprot.readString()
+            self.group_names.append(_elem871)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27898,8 +27959,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter865 in self.group_names:
-        oprot.writeString(iter865)
+      for iter872 in self.group_names:
+        oprot.writeString(iter872)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27954,10 +28015,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype869, _size866) = iprot.readListBegin()
-          for _i870 in xrange(_size866):
-            _elem871 = iprot.readString()
-            self.success.append(_elem871)
+          (_etype876, _size873) = iprot.readListBegin()
+          for _i877 in xrange(_size873):
+            _elem878 = iprot.readString()
+            self.success.append(_elem878)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27980,8 +28041,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter872 in self.success:
-        oprot.writeString(iter872)
+      for iter879 in self.success:
+        oprot.writeString(iter879)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28927,6 +28988,138 @@ class abort_txn_result:
   def __ne__(self, other):
     return not (self == other)
 
+class abort_txns_args:
+  """
+  Attributes:
+   - rqst
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'rqst', (AbortTxnsRequest, AbortTxnsRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, rqst=None,):
+    self.rqst = rqst
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.rqst = AbortTxnsRequest()
+          self.rqst.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('abort_txns_args')
+    if self.rqst is not None:
+      oprot.writeFieldBegin('rqst', TType.STRUCT, 1)
+      self.rqst.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.rqst)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class abort_txns_result:
+  """
+  Attributes:
+   - o1
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'o1', (NoSuchTxnException, NoSuchTxnException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, o1=None,):
+    self.o1 = o1
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = NoSuchTxnException()
+          self.o1.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('abort_txns_result')
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.o1)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class commit_txn_args:
   """
   Attributes:


[3/8] hive git commit: HIVE-12634 : Add command to kill an ACID transacton (Wei Zheng, reviewed by Eugene Koifman)

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/aecb0c02/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 6154d8c..438e368 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -948,6 +948,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function abort_txn(\metastore\AbortTxnRequest $rqst);
   /**
+   * @param \metastore\AbortTxnsRequest $rqst
+   * @throws \metastore\NoSuchTxnException
+   */
+  public function abort_txns(\metastore\AbortTxnsRequest $rqst);
+  /**
    * @param \metastore\CommitTxnRequest $rqst
    * @throws \metastore\NoSuchTxnException
    * @throws \metastore\TxnAbortedException
@@ -7532,6 +7537,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function abort_txns(\metastore\AbortTxnsRequest $rqst)
+  {
+    $this->send_abort_txns($rqst);
+    $this->recv_abort_txns();
+  }
+
+  public function send_abort_txns(\metastore\AbortTxnsRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_abort_txns_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'abort_txns', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('abort_txns', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_abort_txns()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_abort_txns_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_abort_txns_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    return;
+  }
+
   public function commit_txn(\metastore\CommitTxnRequest $rqst)
   {
     $this->send_commit_txn($rqst);
@@ -9372,14 +9428,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size472 = 0;
-            $_etype475 = 0;
-            $xfer += $input->readListBegin($_etype475, $_size472);
-            for ($_i476 = 0; $_i476 < $_size472; ++$_i476)
+            $_size479 = 0;
+            $_etype482 = 0;
+            $xfer += $input->readListBegin($_etype482, $_size479);
+            for ($_i483 = 0; $_i483 < $_size479; ++$_i483)
             {
-              $elem477 = null;
-              $xfer += $input->readString($elem477);
-              $this->success []= $elem477;
+              $elem484 = null;
+              $xfer += $input->readString($elem484);
+              $this->success []= $elem484;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9415,9 +9471,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter478)
+          foreach ($this->success as $iter485)
           {
-            $xfer += $output->writeString($iter478);
+            $xfer += $output->writeString($iter485);
           }
         }
         $output->writeListEnd();
@@ -9548,14 +9604,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size479 = 0;
-            $_etype482 = 0;
-            $xfer += $input->readListBegin($_etype482, $_size479);
-            for ($_i483 = 0; $_i483 < $_size479; ++$_i483)
+            $_size486 = 0;
+            $_etype489 = 0;
+            $xfer += $input->readListBegin($_etype489, $_size486);
+            for ($_i490 = 0; $_i490 < $_size486; ++$_i490)
             {
-              $elem484 = null;
-              $xfer += $input->readString($elem484);
-              $this->success []= $elem484;
+              $elem491 = null;
+              $xfer += $input->readString($elem491);
+              $this->success []= $elem491;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -9591,9 +9647,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter485)
+          foreach ($this->success as $iter492)
           {
-            $xfer += $output->writeString($iter485);
+            $xfer += $output->writeString($iter492);
           }
         }
         $output->writeListEnd();
@@ -10594,18 +10650,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size486 = 0;
-            $_ktype487 = 0;
-            $_vtype488 = 0;
-            $xfer += $input->readMapBegin($_ktype487, $_vtype488, $_size486);
-            for ($_i490 = 0; $_i490 < $_size486; ++$_i490)
+            $_size493 = 0;
+            $_ktype494 = 0;
+            $_vtype495 = 0;
+            $xfer += $input->readMapBegin($_ktype494, $_vtype495, $_size493);
+            for ($_i497 = 0; $_i497 < $_size493; ++$_i497)
             {
-              $key491 = '';
-              $val492 = new \metastore\Type();
-              $xfer += $input->readString($key491);
-              $val492 = new \metastore\Type();
-              $xfer += $val492->read($input);
-              $this->success[$key491] = $val492;
+              $key498 = '';
+              $val499 = new \metastore\Type();
+              $xfer += $input->readString($key498);
+              $val499 = new \metastore\Type();
+              $xfer += $val499->read($input);
+              $this->success[$key498] = $val499;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -10641,10 +10697,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter493 => $viter494)
+          foreach ($this->success as $kiter500 => $viter501)
           {
-            $xfer += $output->writeString($kiter493);
-            $xfer += $viter494->write($output);
+            $xfer += $output->writeString($kiter500);
+            $xfer += $viter501->write($output);
           }
         }
         $output->writeMapEnd();
@@ -10848,15 +10904,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size495 = 0;
-            $_etype498 = 0;
-            $xfer += $input->readListBegin($_etype498, $_size495);
-            for ($_i499 = 0; $_i499 < $_size495; ++$_i499)
+            $_size502 = 0;
+            $_etype505 = 0;
+            $xfer += $input->readListBegin($_etype505, $_size502);
+            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
             {
-              $elem500 = null;
-              $elem500 = new \metastore\FieldSchema();
-              $xfer += $elem500->read($input);
-              $this->success []= $elem500;
+              $elem507 = null;
+              $elem507 = new \metastore\FieldSchema();
+              $xfer += $elem507->read($input);
+              $this->success []= $elem507;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -10908,9 +10964,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter501)
+          foreach ($this->success as $iter508)
           {
-            $xfer += $iter501->write($output);
+            $xfer += $iter508->write($output);
           }
         }
         $output->writeListEnd();
@@ -11152,15 +11208,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size502 = 0;
-            $_etype505 = 0;
-            $xfer += $input->readListBegin($_etype505, $_size502);
-            for ($_i506 = 0; $_i506 < $_size502; ++$_i506)
+            $_size509 = 0;
+            $_etype512 = 0;
+            $xfer += $input->readListBegin($_etype512, $_size509);
+            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
             {
-              $elem507 = null;
-              $elem507 = new \metastore\FieldSchema();
-              $xfer += $elem507->read($input);
-              $this->success []= $elem507;
+              $elem514 = null;
+              $elem514 = new \metastore\FieldSchema();
+              $xfer += $elem514->read($input);
+              $this->success []= $elem514;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11212,9 +11268,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter508)
+          foreach ($this->success as $iter515)
           {
-            $xfer += $iter508->write($output);
+            $xfer += $iter515->write($output);
           }
         }
         $output->writeListEnd();
@@ -11428,15 +11484,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size509 = 0;
-            $_etype512 = 0;
-            $xfer += $input->readListBegin($_etype512, $_size509);
-            for ($_i513 = 0; $_i513 < $_size509; ++$_i513)
+            $_size516 = 0;
+            $_etype519 = 0;
+            $xfer += $input->readListBegin($_etype519, $_size516);
+            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
             {
-              $elem514 = null;
-              $elem514 = new \metastore\FieldSchema();
-              $xfer += $elem514->read($input);
-              $this->success []= $elem514;
+              $elem521 = null;
+              $elem521 = new \metastore\FieldSchema();
+              $xfer += $elem521->read($input);
+              $this->success []= $elem521;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11488,9 +11544,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter515)
+          foreach ($this->success as $iter522)
           {
-            $xfer += $iter515->write($output);
+            $xfer += $iter522->write($output);
           }
         }
         $output->writeListEnd();
@@ -11732,15 +11788,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size516 = 0;
-            $_etype519 = 0;
-            $xfer += $input->readListBegin($_etype519, $_size516);
-            for ($_i520 = 0; $_i520 < $_size516; ++$_i520)
+            $_size523 = 0;
+            $_etype526 = 0;
+            $xfer += $input->readListBegin($_etype526, $_size523);
+            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
             {
-              $elem521 = null;
-              $elem521 = new \metastore\FieldSchema();
-              $xfer += $elem521->read($input);
-              $this->success []= $elem521;
+              $elem528 = null;
+              $elem528 = new \metastore\FieldSchema();
+              $xfer += $elem528->read($input);
+              $this->success []= $elem528;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -11792,9 +11848,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter522)
+          foreach ($this->success as $iter529)
           {
-            $xfer += $iter522->write($output);
+            $xfer += $iter529->write($output);
           }
         }
         $output->writeListEnd();
@@ -12949,14 +13005,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size523 = 0;
-            $_etype526 = 0;
-            $xfer += $input->readListBegin($_etype526, $_size523);
-            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            $_size530 = 0;
+            $_etype533 = 0;
+            $xfer += $input->readListBegin($_etype533, $_size530);
+            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
             {
-              $elem528 = null;
-              $xfer += $input->readString($elem528);
-              $this->success []= $elem528;
+              $elem535 = null;
+              $xfer += $input->readString($elem535);
+              $this->success []= $elem535;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -12992,9 +13048,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter529)
+          foreach ($this->success as $iter536)
           {
-            $xfer += $output->writeString($iter529);
+            $xfer += $output->writeString($iter536);
           }
         }
         $output->writeListEnd();
@@ -13150,14 +13206,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size530 = 0;
-            $_etype533 = 0;
-            $xfer += $input->readListBegin($_etype533, $_size530);
-            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
+            $_size537 = 0;
+            $_etype540 = 0;
+            $xfer += $input->readListBegin($_etype540, $_size537);
+            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
             {
-              $elem535 = null;
-              $xfer += $input->readString($elem535);
-              $this->success []= $elem535;
+              $elem542 = null;
+              $xfer += $input->readString($elem542);
+              $this->success []= $elem542;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13193,9 +13249,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter536)
+          foreach ($this->success as $iter543)
           {
-            $xfer += $output->writeString($iter536);
+            $xfer += $output->writeString($iter543);
           }
         }
         $output->writeListEnd();
@@ -13510,14 +13566,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size537 = 0;
-            $_etype540 = 0;
-            $xfer += $input->readListBegin($_etype540, $_size537);
-            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
+            $_size544 = 0;
+            $_etype547 = 0;
+            $xfer += $input->readListBegin($_etype547, $_size544);
+            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
             {
-              $elem542 = null;
-              $xfer += $input->readString($elem542);
-              $this->tbl_names []= $elem542;
+              $elem549 = null;
+              $xfer += $input->readString($elem549);
+              $this->tbl_names []= $elem549;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13550,9 +13606,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter543)
+          foreach ($this->tbl_names as $iter550)
           {
-            $xfer += $output->writeString($iter543);
+            $xfer += $output->writeString($iter550);
           }
         }
         $output->writeListEnd();
@@ -13653,15 +13709,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size544 = 0;
-            $_etype547 = 0;
-            $xfer += $input->readListBegin($_etype547, $_size544);
-            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
+            $_size551 = 0;
+            $_etype554 = 0;
+            $xfer += $input->readListBegin($_etype554, $_size551);
+            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
             {
-              $elem549 = null;
-              $elem549 = new \metastore\Table();
-              $xfer += $elem549->read($input);
-              $this->success []= $elem549;
+              $elem556 = null;
+              $elem556 = new \metastore\Table();
+              $xfer += $elem556->read($input);
+              $this->success []= $elem556;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -13713,9 +13769,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter550)
+          foreach ($this->success as $iter557)
           {
-            $xfer += $iter550->write($output);
+            $xfer += $iter557->write($output);
           }
         }
         $output->writeListEnd();
@@ -13951,14 +14007,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size558 = 0;
+            $_etype561 = 0;
+            $xfer += $input->readListBegin($_etype561, $_size558);
+            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
             {
-              $elem556 = null;
-              $xfer += $input->readString($elem556);
-              $this->success []= $elem556;
+              $elem563 = null;
+              $xfer += $input->readString($elem563);
+              $this->success []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -14010,9 +14066,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter557)
+          foreach ($this->success as $iter564)
           {
-            $xfer += $output->writeString($iter557);
+            $xfer += $output->writeString($iter564);
           }
         }
         $output->writeListEnd();
@@ -15325,15 +15381,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size558 = 0;
-            $_etype561 = 0;
-            $xfer += $input->readListBegin($_etype561, $_size558);
-            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
+            $_size565 = 0;
+            $_etype568 = 0;
+            $xfer += $input->readListBegin($_etype568, $_size565);
+            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
             {
-              $elem563 = null;
-              $elem563 = new \metastore\Partition();
-              $xfer += $elem563->read($input);
-              $this->new_parts []= $elem563;
+              $elem570 = null;
+              $elem570 = new \metastore\Partition();
+              $xfer += $elem570->read($input);
+              $this->new_parts []= $elem570;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15361,9 +15417,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter564)
+          foreach ($this->new_parts as $iter571)
           {
-            $xfer += $iter564->write($output);
+            $xfer += $iter571->write($output);
           }
         }
         $output->writeListEnd();
@@ -15578,15 +15634,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size572 = 0;
+            $_etype575 = 0;
+            $xfer += $input->readListBegin($_etype575, $_size572);
+            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\PartitionSpec();
-              $xfer += $elem570->read($input);
-              $this->new_parts []= $elem570;
+              $elem577 = null;
+              $elem577 = new \metastore\PartitionSpec();
+              $xfer += $elem577->read($input);
+              $this->new_parts []= $elem577;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15614,9 +15670,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter571)
+          foreach ($this->new_parts as $iter578)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter578->write($output);
           }
         }
         $output->writeListEnd();
@@ -15866,14 +15922,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size579 = 0;
+            $_etype582 = 0;
+            $xfer += $input->readListBegin($_etype582, $_size579);
+            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
             {
-              $elem577 = null;
-              $xfer += $input->readString($elem577);
-              $this->part_vals []= $elem577;
+              $elem584 = null;
+              $xfer += $input->readString($elem584);
+              $this->part_vals []= $elem584;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15911,9 +15967,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter578)
+          foreach ($this->part_vals as $iter585)
           {
-            $xfer += $output->writeString($iter578);
+            $xfer += $output->writeString($iter585);
           }
         }
         $output->writeListEnd();
@@ -16415,14 +16471,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size579 = 0;
-            $_etype582 = 0;
-            $xfer += $input->readListBegin($_etype582, $_size579);
-            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
+            $_size586 = 0;
+            $_etype589 = 0;
+            $xfer += $input->readListBegin($_etype589, $_size586);
+            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
             {
-              $elem584 = null;
-              $xfer += $input->readString($elem584);
-              $this->part_vals []= $elem584;
+              $elem591 = null;
+              $xfer += $input->readString($elem591);
+              $this->part_vals []= $elem591;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16468,9 +16524,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter585)
+          foreach ($this->part_vals as $iter592)
           {
-            $xfer += $output->writeString($iter585);
+            $xfer += $output->writeString($iter592);
           }
         }
         $output->writeListEnd();
@@ -17324,14 +17380,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readListBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size593 = 0;
+            $_etype596 = 0;
+            $xfer += $input->readListBegin($_etype596, $_size593);
+            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
             {
-              $elem591 = null;
-              $xfer += $input->readString($elem591);
-              $this->part_vals []= $elem591;
+              $elem598 = null;
+              $xfer += $input->readString($elem598);
+              $this->part_vals []= $elem598;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17376,9 +17432,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter592)
+          foreach ($this->part_vals as $iter599)
           {
-            $xfer += $output->writeString($iter592);
+            $xfer += $output->writeString($iter599);
           }
         }
         $output->writeListEnd();
@@ -17631,14 +17687,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size593 = 0;
-            $_etype596 = 0;
-            $xfer += $input->readListBegin($_etype596, $_size593);
-            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
+            $_size600 = 0;
+            $_etype603 = 0;
+            $xfer += $input->readListBegin($_etype603, $_size600);
+            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
             {
-              $elem598 = null;
-              $xfer += $input->readString($elem598);
-              $this->part_vals []= $elem598;
+              $elem605 = null;
+              $xfer += $input->readString($elem605);
+              $this->part_vals []= $elem605;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17691,9 +17747,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter599)
+          foreach ($this->part_vals as $iter606)
           {
-            $xfer += $output->writeString($iter599);
+            $xfer += $output->writeString($iter606);
           }
         }
         $output->writeListEnd();
@@ -18707,14 +18763,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size600 = 0;
-            $_etype603 = 0;
-            $xfer += $input->readListBegin($_etype603, $_size600);
-            for ($_i604 = 0; $_i604 < $_size600; ++$_i604)
+            $_size607 = 0;
+            $_etype610 = 0;
+            $xfer += $input->readListBegin($_etype610, $_size607);
+            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
             {
-              $elem605 = null;
-              $xfer += $input->readString($elem605);
-              $this->part_vals []= $elem605;
+              $elem612 = null;
+              $xfer += $input->readString($elem612);
+              $this->part_vals []= $elem612;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18752,9 +18808,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter606)
+          foreach ($this->part_vals as $iter613)
           {
-            $xfer += $output->writeString($iter606);
+            $xfer += $output->writeString($iter613);
           }
         }
         $output->writeListEnd();
@@ -18996,17 +19052,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size607 = 0;
-            $_ktype608 = 0;
-            $_vtype609 = 0;
-            $xfer += $input->readMapBegin($_ktype608, $_vtype609, $_size607);
-            for ($_i611 = 0; $_i611 < $_size607; ++$_i611)
+            $_size614 = 0;
+            $_ktype615 = 0;
+            $_vtype616 = 0;
+            $xfer += $input->readMapBegin($_ktype615, $_vtype616, $_size614);
+            for ($_i618 = 0; $_i618 < $_size614; ++$_i618)
             {
-              $key612 = '';
-              $val613 = '';
-              $xfer += $input->readString($key612);
-              $xfer += $input->readString($val613);
-              $this->partitionSpecs[$key612] = $val613;
+              $key619 = '';
+              $val620 = '';
+              $xfer += $input->readString($key619);
+              $xfer += $input->readString($val620);
+              $this->partitionSpecs[$key619] = $val620;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19062,10 +19118,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter614 => $viter615)
+          foreach ($this->partitionSpecs as $kiter621 => $viter622)
           {
-            $xfer += $output->writeString($kiter614);
-            $xfer += $output->writeString($viter615);
+            $xfer += $output->writeString($kiter621);
+            $xfer += $output->writeString($viter622);
           }
         }
         $output->writeMapEnd();
@@ -19377,17 +19433,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size616 = 0;
-            $_ktype617 = 0;
-            $_vtype618 = 0;
-            $xfer += $input->readMapBegin($_ktype617, $_vtype618, $_size616);
-            for ($_i620 = 0; $_i620 < $_size616; ++$_i620)
+            $_size623 = 0;
+            $_ktype624 = 0;
+            $_vtype625 = 0;
+            $xfer += $input->readMapBegin($_ktype624, $_vtype625, $_size623);
+            for ($_i627 = 0; $_i627 < $_size623; ++$_i627)
             {
-              $key621 = '';
-              $val622 = '';
-              $xfer += $input->readString($key621);
-              $xfer += $input->readString($val622);
-              $this->partitionSpecs[$key621] = $val622;
+              $key628 = '';
+              $val629 = '';
+              $xfer += $input->readString($key628);
+              $xfer += $input->readString($val629);
+              $this->partitionSpecs[$key628] = $val629;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19443,10 +19499,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter623 => $viter624)
+          foreach ($this->partitionSpecs as $kiter630 => $viter631)
           {
-            $xfer += $output->writeString($kiter623);
-            $xfer += $output->writeString($viter624);
+            $xfer += $output->writeString($kiter630);
+            $xfer += $output->writeString($viter631);
           }
         }
         $output->writeMapEnd();
@@ -19579,15 +19635,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readListBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readListBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem630 = null;
-              $elem630 = new \metastore\Partition();
-              $xfer += $elem630->read($input);
-              $this->success []= $elem630;
+              $elem637 = null;
+              $elem637 = new \metastore\Partition();
+              $xfer += $elem637->read($input);
+              $this->success []= $elem637;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19647,9 +19703,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter631)
+          foreach ($this->success as $iter638)
           {
-            $xfer += $iter631->write($output);
+            $xfer += $iter638->write($output);
           }
         }
         $output->writeListEnd();
@@ -19795,14 +19851,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size632 = 0;
-            $_etype635 = 0;
-            $xfer += $input->readListBegin($_etype635, $_size632);
-            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
+            $_size639 = 0;
+            $_etype642 = 0;
+            $xfer += $input->readListBegin($_etype642, $_size639);
+            for ($_i643 = 0; $_i643 < $_size639; ++$_i643)
             {
-              $elem637 = null;
-              $xfer += $input->readString($elem637);
-              $this->part_vals []= $elem637;
+              $elem644 = null;
+              $xfer += $input->readString($elem644);
+              $this->part_vals []= $elem644;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19819,14 +19875,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size638 = 0;
-            $_etype641 = 0;
-            $xfer += $input->readListBegin($_etype641, $_size638);
-            for ($_i642 = 0; $_i642 < $_size638; ++$_i642)
+            $_size645 = 0;
+            $_etype648 = 0;
+            $xfer += $input->readListBegin($_etype648, $_size645);
+            for ($_i649 = 0; $_i649 < $_size645; ++$_i649)
             {
-              $elem643 = null;
-              $xfer += $input->readString($elem643);
-              $this->group_names []= $elem643;
+              $elem650 = null;
+              $xfer += $input->readString($elem650);
+              $this->group_names []= $elem650;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19864,9 +19920,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter644)
+          foreach ($this->part_vals as $iter651)
           {
-            $xfer += $output->writeString($iter644);
+            $xfer += $output->writeString($iter651);
           }
         }
         $output->writeListEnd();
@@ -19886,9 +19942,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter645)
+          foreach ($this->group_names as $iter652)
           {
-            $xfer += $output->writeString($iter645);
+            $xfer += $output->writeString($iter652);
           }
         }
         $output->writeListEnd();
@@ -20479,15 +20535,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
             {
-              $elem651 = null;
-              $elem651 = new \metastore\Partition();
-              $xfer += $elem651->read($input);
-              $this->success []= $elem651;
+              $elem658 = null;
+              $elem658 = new \metastore\Partition();
+              $xfer += $elem658->read($input);
+              $this->success []= $elem658;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20531,9 +20587,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter652)
+          foreach ($this->success as $iter659)
           {
-            $xfer += $iter652->write($output);
+            $xfer += $iter659->write($output);
           }
         }
         $output->writeListEnd();
@@ -20679,14 +20735,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size653 = 0;
-            $_etype656 = 0;
-            $xfer += $input->readListBegin($_etype656, $_size653);
-            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            $_size660 = 0;
+            $_etype663 = 0;
+            $xfer += $input->readListBegin($_etype663, $_size660);
+            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
             {
-              $elem658 = null;
-              $xfer += $input->readString($elem658);
-              $this->group_names []= $elem658;
+              $elem665 = null;
+              $xfer += $input->readString($elem665);
+              $this->group_names []= $elem665;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20734,9 +20790,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter659)
+          foreach ($this->group_names as $iter666)
           {
-            $xfer += $output->writeString($iter659);
+            $xfer += $output->writeString($iter666);
           }
         }
         $output->writeListEnd();
@@ -20825,15 +20881,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size660 = 0;
-            $_etype663 = 0;
-            $xfer += $input->readListBegin($_etype663, $_size660);
-            for ($_i664 = 0; $_i664 < $_size660; ++$_i664)
+            $_size667 = 0;
+            $_etype670 = 0;
+            $xfer += $input->readListBegin($_etype670, $_size667);
+            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
             {
-              $elem665 = null;
-              $elem665 = new \metastore\Partition();
-              $xfer += $elem665->read($input);
-              $this->success []= $elem665;
+              $elem672 = null;
+              $elem672 = new \metastore\Partition();
+              $xfer += $elem672->read($input);
+              $this->success []= $elem672;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20877,9 +20933,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter666)
+          foreach ($this->success as $iter673)
           {
-            $xfer += $iter666->write($output);
+            $xfer += $iter673->write($output);
           }
         }
         $output->writeListEnd();
@@ -21099,15 +21155,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size667 = 0;
-            $_etype670 = 0;
-            $xfer += $input->readListBegin($_etype670, $_size667);
-            for ($_i671 = 0; $_i671 < $_size667; ++$_i671)
+            $_size674 = 0;
+            $_etype677 = 0;
+            $xfer += $input->readListBegin($_etype677, $_size674);
+            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
             {
-              $elem672 = null;
-              $elem672 = new \metastore\PartitionSpec();
-              $xfer += $elem672->read($input);
-              $this->success []= $elem672;
+              $elem679 = null;
+              $elem679 = new \metastore\PartitionSpec();
+              $xfer += $elem679->read($input);
+              $this->success []= $elem679;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21151,9 +21207,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter673)
+          foreach ($this->success as $iter680)
           {
-            $xfer += $iter673->write($output);
+            $xfer += $iter680->write($output);
           }
         }
         $output->writeListEnd();
@@ -21360,14 +21416,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size674 = 0;
-            $_etype677 = 0;
-            $xfer += $input->readListBegin($_etype677, $_size674);
-            for ($_i678 = 0; $_i678 < $_size674; ++$_i678)
+            $_size681 = 0;
+            $_etype684 = 0;
+            $xfer += $input->readListBegin($_etype684, $_size681);
+            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
             {
-              $elem679 = null;
-              $xfer += $input->readString($elem679);
-              $this->success []= $elem679;
+              $elem686 = null;
+              $xfer += $input->readString($elem686);
+              $this->success []= $elem686;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21403,9 +21459,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter680)
+          foreach ($this->success as $iter687)
           {
-            $xfer += $output->writeString($iter680);
+            $xfer += $output->writeString($iter687);
           }
         }
         $output->writeListEnd();
@@ -21521,14 +21577,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size681 = 0;
-            $_etype684 = 0;
-            $xfer += $input->readListBegin($_etype684, $_size681);
-            for ($_i685 = 0; $_i685 < $_size681; ++$_i685)
+            $_size688 = 0;
+            $_etype691 = 0;
+            $xfer += $input->readListBegin($_etype691, $_size688);
+            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
             {
-              $elem686 = null;
-              $xfer += $input->readString($elem686);
-              $this->part_vals []= $elem686;
+              $elem693 = null;
+              $xfer += $input->readString($elem693);
+              $this->part_vals []= $elem693;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21573,9 +21629,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter687)
+          foreach ($this->part_vals as $iter694)
           {
-            $xfer += $output->writeString($iter687);
+            $xfer += $output->writeString($iter694);
           }
         }
         $output->writeListEnd();
@@ -21669,15 +21725,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size688 = 0;
-            $_etype691 = 0;
-            $xfer += $input->readListBegin($_etype691, $_size688);
-            for ($_i692 = 0; $_i692 < $_size688; ++$_i692)
+            $_size695 = 0;
+            $_etype698 = 0;
+            $xfer += $input->readListBegin($_etype698, $_size695);
+            for ($_i699 = 0; $_i699 < $_size695; ++$_i699)
             {
-              $elem693 = null;
-              $elem693 = new \metastore\Partition();
-              $xfer += $elem693->read($input);
-              $this->success []= $elem693;
+              $elem700 = null;
+              $elem700 = new \metastore\Partition();
+              $xfer += $elem700->read($input);
+              $this->success []= $elem700;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21721,9 +21777,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter694)
+          foreach ($this->success as $iter701)
           {
-            $xfer += $iter694->write($output);
+            $xfer += $iter701->write($output);
           }
         }
         $output->writeListEnd();
@@ -21870,14 +21926,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size695 = 0;
-            $_etype698 = 0;
-            $xfer += $input->readListBegin($_etype698, $_size695);
-            for ($_i699 = 0; $_i699 < $_size695; ++$_i699)
+            $_size702 = 0;
+            $_etype705 = 0;
+            $xfer += $input->readListBegin($_etype705, $_size702);
+            for ($_i706 = 0; $_i706 < $_size702; ++$_i706)
             {
-              $elem700 = null;
-              $xfer += $input->readString($elem700);
-              $this->part_vals []= $elem700;
+              $elem707 = null;
+              $xfer += $input->readString($elem707);
+              $this->part_vals []= $elem707;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21901,14 +21957,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size701 = 0;
-            $_etype704 = 0;
-            $xfer += $input->readListBegin($_etype704, $_size701);
-            for ($_i705 = 0; $_i705 < $_size701; ++$_i705)
+            $_size708 = 0;
+            $_etype711 = 0;
+            $xfer += $input->readListBegin($_etype711, $_size708);
+            for ($_i712 = 0; $_i712 < $_size708; ++$_i712)
             {
-              $elem706 = null;
-              $xfer += $input->readString($elem706);
-              $this->group_names []= $elem706;
+              $elem713 = null;
+              $xfer += $input->readString($elem713);
+              $this->group_names []= $elem713;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21946,9 +22002,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter707)
+          foreach ($this->part_vals as $iter714)
           {
-            $xfer += $output->writeString($iter707);
+            $xfer += $output->writeString($iter714);
           }
         }
         $output->writeListEnd();
@@ -21973,9 +22029,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter708)
+          foreach ($this->group_names as $iter715)
           {
-            $xfer += $output->writeString($iter708);
+            $xfer += $output->writeString($iter715);
           }
         }
         $output->writeListEnd();
@@ -22064,15 +22120,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size709 = 0;
-            $_etype712 = 0;
-            $xfer += $input->readListBegin($_etype712, $_size709);
-            for ($_i713 = 0; $_i713 < $_size709; ++$_i713)
+            $_size716 = 0;
+            $_etype719 = 0;
+            $xfer += $input->readListBegin($_etype719, $_size716);
+            for ($_i720 = 0; $_i720 < $_size716; ++$_i720)
             {
-              $elem714 = null;
-              $elem714 = new \metastore\Partition();
-              $xfer += $elem714->read($input);
-              $this->success []= $elem714;
+              $elem721 = null;
+              $elem721 = new \metastore\Partition();
+              $xfer += $elem721->read($input);
+              $this->success []= $elem721;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22116,9 +22172,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter715)
+          foreach ($this->success as $iter722)
           {
-            $xfer += $iter715->write($output);
+            $xfer += $iter722->write($output);
           }
         }
         $output->writeListEnd();
@@ -22239,14 +22295,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size716 = 0;
-            $_etype719 = 0;
-            $xfer += $input->readListBegin($_etype719, $_size716);
-            for ($_i720 = 0; $_i720 < $_size716; ++$_i720)
+            $_size723 = 0;
+            $_etype726 = 0;
+            $xfer += $input->readListBegin($_etype726, $_size723);
+            for ($_i727 = 0; $_i727 < $_size723; ++$_i727)
             {
-              $elem721 = null;
-              $xfer += $input->readString($elem721);
-              $this->part_vals []= $elem721;
+              $elem728 = null;
+              $xfer += $input->readString($elem728);
+              $this->part_vals []= $elem728;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22291,9 +22347,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter722)
+          foreach ($this->part_vals as $iter729)
           {
-            $xfer += $output->writeString($iter722);
+            $xfer += $output->writeString($iter729);
           }
         }
         $output->writeListEnd();
@@ -22386,14 +22442,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size723 = 0;
-            $_etype726 = 0;
-            $xfer += $input->readListBegin($_etype726, $_size723);
-            for ($_i727 = 0; $_i727 < $_size723; ++$_i727)
+            $_size730 = 0;
+            $_etype733 = 0;
+            $xfer += $input->readListBegin($_etype733, $_size730);
+            for ($_i734 = 0; $_i734 < $_size730; ++$_i734)
             {
-              $elem728 = null;
-              $xfer += $input->readString($elem728);
-              $this->success []= $elem728;
+              $elem735 = null;
+              $xfer += $input->readString($elem735);
+              $this->success []= $elem735;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22437,9 +22493,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter729)
+          foreach ($this->success as $iter736)
           {
-            $xfer += $output->writeString($iter729);
+            $xfer += $output->writeString($iter736);
           }
         }
         $output->writeListEnd();
@@ -22682,15 +22738,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size730 = 0;
-            $_etype733 = 0;
-            $xfer += $input->readListBegin($_etype733, $_size730);
-            for ($_i734 = 0; $_i734 < $_size730; ++$_i734)
+            $_size737 = 0;
+            $_etype740 = 0;
+            $xfer += $input->readListBegin($_etype740, $_size737);
+            for ($_i741 = 0; $_i741 < $_size737; ++$_i741)
             {
-              $elem735 = null;
-              $elem735 = new \metastore\Partition();
-              $xfer += $elem735->read($input);
-              $this->success []= $elem735;
+              $elem742 = null;
+              $elem742 = new \metastore\Partition();
+              $xfer += $elem742->read($input);
+              $this->success []= $elem742;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22734,9 +22790,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter736)
+          foreach ($this->success as $iter743)
           {
-            $xfer += $iter736->write($output);
+            $xfer += $iter743->write($output);
           }
         }
         $output->writeListEnd();
@@ -22979,15 +23035,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size737 = 0;
-            $_etype740 = 0;
-            $xfer += $input->readListBegin($_etype740, $_size737);
-            for ($_i741 = 0; $_i741 < $_size737; ++$_i741)
+            $_size744 = 0;
+            $_etype747 = 0;
+            $xfer += $input->readListBegin($_etype747, $_size744);
+            for ($_i748 = 0; $_i748 < $_size744; ++$_i748)
             {
-              $elem742 = null;
-              $elem742 = new \metastore\PartitionSpec();
-              $xfer += $elem742->read($input);
-              $this->success []= $elem742;
+              $elem749 = null;
+              $elem749 = new \metastore\PartitionSpec();
+              $xfer += $elem749->read($input);
+              $this->success []= $elem749;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23031,9 +23087,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter743)
+          foreach ($this->success as $iter750)
           {
-            $xfer += $iter743->write($output);
+            $xfer += $iter750->write($output);
           }
         }
         $output->writeListEnd();
@@ -23353,14 +23409,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size744 = 0;
-            $_etype747 = 0;
-            $xfer += $input->readListBegin($_etype747, $_size744);
-            for ($_i748 = 0; $_i748 < $_size744; ++$_i748)
+            $_size751 = 0;
+            $_etype754 = 0;
+            $xfer += $input->readListBegin($_etype754, $_size751);
+            for ($_i755 = 0; $_i755 < $_size751; ++$_i755)
             {
-              $elem749 = null;
-              $xfer += $input->readString($elem749);
-              $this->names []= $elem749;
+              $elem756 = null;
+              $xfer += $input->readString($elem756);
+              $this->names []= $elem756;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23398,9 +23454,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter750)
+          foreach ($this->names as $iter757)
           {
-            $xfer += $output->writeString($iter750);
+            $xfer += $output->writeString($iter757);
           }
         }
         $output->writeListEnd();
@@ -23489,15 +23545,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size751 = 0;
-            $_etype754 = 0;
-            $xfer += $input->readListBegin($_etype754, $_size751);
-            for ($_i755 = 0; $_i755 < $_size751; ++$_i755)
+            $_size758 = 0;
+            $_etype761 = 0;
+            $xfer += $input->readListBegin($_etype761, $_size758);
+            for ($_i762 = 0; $_i762 < $_size758; ++$_i762)
             {
-              $elem756 = null;
-              $elem756 = new \metastore\Partition();
-              $xfer += $elem756->read($input);
-              $this->success []= $elem756;
+              $elem763 = null;
+              $elem763 = new \metastore\Partition();
+              $xfer += $elem763->read($input);
+              $this->success []= $elem763;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23541,9 +23597,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter757)
+          foreach ($this->success as $iter764)
           {
-            $xfer += $iter757->write($output);
+            $xfer += $iter764->write($output);
           }
         }
         $output->writeListEnd();
@@ -23882,15 +23938,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size758 = 0;
-            $_etype761 = 0;
-            $xfer += $input->readListBegin($_etype761, $_size758);
-            for ($_i762 = 0; $_i762 < $_size758; ++$_i762)
+            $_size765 = 0;
+            $_etype768 = 0;
+            $xfer += $input->readListBegin($_etype768, $_size765);
+            for ($_i769 = 0; $_i769 < $_size765; ++$_i769)
             {
-              $elem763 = null;
-              $elem763 = new \metastore\Partition();
-              $xfer += $elem763->read($input);
-              $this->new_parts []= $elem763;
+              $elem770 = null;
+              $elem770 = new \metastore\Partition();
+              $xfer += $elem770->read($input);
+              $this->new_parts []= $elem770;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23928,9 +23984,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter764)
+          foreach ($this->new_parts as $iter771)
           {
-            $xfer += $iter764->write($output);
+            $xfer += $iter771->write($output);
           }
         }
         $output->writeListEnd();
@@ -24400,14 +24456,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size765 = 0;
-            $_etype768 = 0;
-            $xfer += $input->readListBegin($_etype768, $_size765);
-            for ($_i769 = 0; $_i769 < $_size765; ++$_i769)
+            $_size772 = 0;
+            $_etype775 = 0;
+            $xfer += $input->readListBegin($_etype775, $_size772);
+            for ($_i776 = 0; $_i776 < $_size772; ++$_i776)
             {
-              $elem770 = null;
-              $xfer += $input->readString($elem770);
-              $this->part_vals []= $elem770;
+              $elem777 = null;
+              $xfer += $input->readString($elem777);
+              $this->part_vals []= $elem777;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24453,9 +24509,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter771)
+          foreach ($this->part_vals as $iter778)
           {
-            $xfer += $output->writeString($iter771);
+            $xfer += $output->writeString($iter778);
           }
         }
         $output->writeListEnd();
@@ -24640,14 +24696,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size772 = 0;
-            $_etype775 = 0;
-            $xfer += $input->readListBegin($_etype775, $_size772);
-            for ($_i776 = 0; $_i776 < $_size772; ++$_i776)
+            $_size779 = 0;
+            $_etype782 = 0;
+            $xfer += $input->readListBegin($_etype782, $_size779);
+            for ($_i783 = 0; $_i783 < $_size779; ++$_i783)
             {
-              $elem777 = null;
-              $xfer += $input->readString($elem777);
-              $this->part_vals []= $elem777;
+              $elem784 = null;
+              $xfer += $input->readString($elem784);
+              $this->part_vals []= $elem784;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24682,9 +24738,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter778)
+          foreach ($this->part_vals as $iter785)
           {
-            $xfer += $output->writeString($iter778);
+            $xfer += $output->writeString($iter785);
           }
         }
         $output->writeListEnd();
@@ -25138,14 +25194,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size779 = 0;
-            $_etype782 = 0;
-            $xfer += $input->readListBegin($_etype782, $_size779);
-            for ($_i783 = 0; $_i783 < $_size779; ++$_i783)
+            $_size786 = 0;
+            $_etype789 = 0;
+            $xfer += $input->readListBegin($_etype789, $_size786);
+            for ($_i790 = 0; $_i790 < $_size786; ++$_i790)
             {
-              $elem784 = null;
-              $xfer += $input->readString($elem784);
-              $this->success []= $elem784;
+              $elem791 = null;
+              $xfer += $input->readString($elem791);
+              $this->success []= $elem791;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25181,9 +25237,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter785)
+          foreach ($this->success as $iter792)
           {
-            $xfer += $output->writeString($iter785);
+            $xfer += $output->writeString($iter792);
           }
         }
         $output->writeListEnd();
@@ -25343,17 +25399,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size786 = 0;
-            $_ktype787 = 0;
-            $_vtype788 = 0;
-            $xfer += $input->readMapBegin($_ktype787, $_vtype788, $_size786);
-            for ($_i790 = 0; $_i790 < $_size786; ++$_i790)
+            $_size793 = 0;
+            $_ktype794 = 0;
+            $_vtype795 = 0;
+            $xfer += $input->readMapBegin($_ktype794, $_vtype795, $_size793);
+            for ($_i797 = 0; $_i797 < $_size793; ++$_i797)
             {
-              $key791 = '';
-              $val792 = '';
-              $xfer += $input->readString($key791);
-              $xfer += $input->readString($val792);
-              $this->success[$key791] = $val792;
+              $key798 = '';
+              $val799 = '';
+              $xfer += $input->readString($key798);
+              $xfer += $input->readString($val799);
+              $this->success[$key798] = $val799;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25389,10 +25445,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter793 => $viter794)
+          foreach ($this->success as $kiter800 => $viter801)
           {
-            $xfer += $output->writeString($kiter793);
-            $xfer += $output->writeString($viter794);
+            $xfer += $output->writeString($kiter800);
+            $xfer += $output->writeString($viter801);
           }
         }
         $output->writeMapEnd();
@@ -25512,17 +25568,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size795 = 0;
-            $_ktype796 = 0;
-            $_vtype797 = 0;
-            $xfer += $input->readMapBegin($_ktype796, $_vtype797, $_size795);
-            for ($_i799 = 0; $_i799 < $_size795; ++$_i799)
+            $_size802 = 0;
+            $_ktype803 = 0;
+            $_vtype804 = 0;
+            $xfer += $input->readMapBegin($_ktype803, $_vtype804, $_size802);
+            for ($_i806 = 0; $_i806 < $_size802; ++$_i806)
             {
-              $key800 = '';
-              $val801 = '';
-              $xfer += $input->readString($key800);
-              $xfer += $input->readString($val801);
-              $this->part_vals[$key800] = $val801;
+              $key807 = '';
+              $val808 = '';
+              $xfer += $input->readString($key807);
+              $xfer += $input->readString($val808);
+              $this->part_vals[$key807] = $val808;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25567,10 +25623,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter802 => $viter803)
+          foreach ($this->part_vals as $kiter809 => $viter810)
           {
-            $xfer += $output->writeString($kiter802);
-            $xfer += $output->writeString($viter803);
+            $xfer += $output->writeString($kiter809);
+            $xfer += $output->writeString($viter810);
           }
         }
         $output->writeMapEnd();
@@ -25892,17 +25948,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size804 = 0;
-            $_ktype805 = 0;
-            $_vtype806 = 0;
-            $xfer += $input->readMapBegin($_ktype805, $_vtype806, $_size804);
-            for ($_i808 = 0; $_i808 < $_size804; ++$_i808)
+            $_size811 = 0;
+            $_ktype812 = 0;
+            $_vtype813 = 0;
+            $xfer += $input->readMapBegin($_ktype812, $_vtype813, $_size811);
+            for ($_i815 = 0; $_i815 < $_size811; ++$_i815)
             {
-              $key809 = '';
-              $val810 = '';
-              $xfer += $input->readString($key809);
-              $xfer += $input->readString($val810);
-              $this->part_vals[$key809] = $val810;
+              $key816 = '';
+              $val817 = '';
+              $xfer += $input->readString($key816);
+              $xfer += $input->readString($val817);
+              $this->part_vals[$key816] = $val817;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -25947,10 +26003,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter811 => $viter812)
+          foreach ($this->part_vals as $kiter818 => $viter819)
           {
-            $xfer += $output->writeString($kiter811);
-            $xfer += $output->writeString($viter812);
+            $xfer += $output->writeString($kiter818);
+            $xfer += $output->writeString($viter819);
           }
         }
         $output->writeMapEnd();
@@ -27424,15 +27480,15 @@ class ThriftHiveMetastore_get_indexes_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\Index();
-              $xfer += $elem818->read($input);
-              $this->success []= $elem818;
+              $elem825 = null;
+              $elem825 = new \metastore\Index();
+              $xfer += $elem825->read($input);
+              $this->success []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27476,9 +27532,9 @@ class ThriftHiveMetastore_get_indexes_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();
@@ -27685,14 +27741,14 @@ class ThriftHiveMetastore_get_index_names_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;
-              $xfer += $input->readString($elem825);
-              $this->success []= $elem825;
+              $elem832 = null;
+              $xfer += $input->readString($elem832);
+              $this->success []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27728,9 +27784,9 @@ class ThriftHiveMetastore_get_index_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter826)
+          foreach ($this->success as $iter833)
           {
-            $xfer += $output->writeString($iter826);
+            $xfer += $output->writeString($iter833);
           }
         }
         $output->writeListEnd();
@@ -31204,14 +31260,14 @@ class ThriftHiveMetastore_get_functions_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 {
@@ -31247,9 +31303,9 @@ class ThriftHiveMetastore_get_functions_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();
@@ -32118,14 +32174,14 @@ class ThriftHiveMetastore_get_role_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size834 = 0;
-            $_etype837 = 0;
-            $xfer += $input->readListBegin($_etype837, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem839 = null;
-              $xfer += $input->readString($elem839);
-              $this->success []= $elem839;
+              $elem846 = null;
+              $xfer += $input->readString($elem846);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32161,9 +32217,9 @@ class ThriftHiveMetastore_get_role_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter840)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $output->writeString($iter840);
+            $xfer += $output->writeString($iter847);
           }
         }
         $output->writeListEnd();
@@ -32854,15 +32910,15 @@ class ThriftHiveMetastore_list_roles_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\Role();
-              $xfer += $elem846->read($input);
-              $this->success []= $elem846;
+              $elem853 = null;
+              $elem853 = new \metastore\Role();
+              $xfer += $elem853->read($input);
+              $this->success []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32898,9 +32954,9 @@ class ThriftHiveMetastore_list_roles_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();
@@ -33562,14 +33618,14 @@ class ThriftHiveMetastore_get_privilege_set_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->group_names = 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->group_names []= $elem853;
+              $elem860 = null;
+              $xfer += $input->readString($elem860);
+              $this->group_names []= $elem860;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33610,9 +33666,9 @@ class ThriftHiveMetastore_get_privilege_set_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter854)
+          foreach ($this->group_names as $iter861)
           {
-            $xfer += $output->writeString($iter854);
+            $xfer += $output->writeString($iter861);
           }
         }
         $output->writeListEnd();
@@ -33920,15 +33976,15 @@ class ThriftHiveMetastore_list_privileges_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size855 = 0;
-            $_etype858 = 0;
-            $xfer += $input->readListBegin($_etype858, $_size855);
-            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
+            $_size862 = 0;
+            $_etype865 = 0;
+            $xfer += $input->readListBegin($_etype865, $_size862);
+            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
             {
-              $elem860 = null;
-              $elem860 = new \metastore\HiveObjectPrivilege();
-              $xfer += $elem860->read($input);
-              $this->success []= $elem860;
+              $elem867 = null;
+              $elem867 = new \metastore\HiveObjectPrivilege();
+              $xfer += $elem867->read($input);
+              $this->success []= $elem867;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33964,9 +34020,9 @@ class ThriftHiveMetastore_list_privileges_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter861)
+          foreach ($this->success as $iter868)
           {
-            $xfer += $iter861->write($output);
+            $xfer += $iter868->write($output);
           }
         }
         $output->writeListEnd();
@@ -34598,14 +34654,14 @@ class ThriftHiveMetastore_set_ugi_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->group_names = 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;
-              $xfer += $input->readString($elem867);
-              $this->group_names []= $elem867;
+              $elem874 = null;
+              $xfer += $input->readString($elem874);
+              $this->group_names []= $elem874;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34638,9 +34694,9 @@ class ThriftHiveMetastore_set_ugi_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter868)
+          foreach ($this->group_names as $iter875)
           {
-            $xfer += $output->writeString($iter868);
+            $xfer += $output->writeString($iter875);
           }
         }
         $output->writeListEnd();
@@ -34716,14 +34772,14 @@ class ThriftHiveMetastore_set_ugi_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = 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->success []= $elem874;
+              $elem881 = null;
+              $xfer += $input->readString($elem881);
+              $this->success []= $elem881;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34759,9 +34815,9 @@ class ThriftHiveMetastore_set_ugi_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter875)
+          foreach ($this->success as $iter882)
           {
-            $xfer += $output->writeString($iter875);
+            $xfer += $output->writeString($iter882);
           }
         }
         $output->writeListEnd();
@@ -35882,6 +35938,163 @@ class ThriftHiveMetastore_abort_txn_result {
 
 }
 
+class ThriftHiveMetastore_abort_txns_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\AbortTxnsRequest
+   */
+  public $rqst = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'rqst',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AbortTxnsRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['rqst'])) {
+        $this->rqst = $vals['rqst'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_abort_txns_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->rqst = new \metastore\AbortTxnsRequest();
+            $xfer += $this->rqst->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_abort_txns_args');
+    if ($this->rqst !== null) {
+      if (!is_object($this->rqst)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1);
+      $xfer += $this->rqst->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_abort_txns_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchTxnException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchTxnException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_abort_txns_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchTxnException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_abort_txns_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ThriftHiveMetastore_commit_txn_args {
   static $_TSPEC;