You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by sa...@apache.org on 2018/07/03 10:02:40 UTC

[01/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Repository: hive
Updated Branches:
  refs/heads/master 285a9b4dc -> f519db7ea


http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 2d87a2f..53c4d24 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -2231,10 +2231,8 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   }
 
   @Override
-  public void replCommitTxn(long srcTxnId, String replPolicy)
+  public void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException {
-    CommitTxnRequest rqst = new CommitTxnRequest(srcTxnId);
-    rqst.setReplPolicy(replPolicy);
     client.commit_txn(rqst);
   }
 
@@ -2475,6 +2473,12 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
     return client.fire_listener_event(rqst);
   }
 
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  @Override
+  public void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.add_write_notification_log(rqst);
+  }
+
   /**
    * Creates a synchronized wrapper for any {@link IMetaStoreClient}.
    * This may be used by multi-threaded applications until we have


[11/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 5c6495e..f5913fc 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -445,6 +445,8 @@ class AbortTxnsRequest;
 
 class CommitTxnRequest;
 
+class WriteEventInfo;
+
 class ReplTblWriteIdStateRequest;
 
 class GetValidWriteIdsRequest;
@@ -517,6 +519,10 @@ class FireEventRequest;
 
 class FireEventResponse;
 
+class WriteNotificationLogRequest;
+
+class WriteNotificationLogResponse;
+
 class MetadataPpdResult;
 
 class GetFileMetadataByExprResult;
@@ -6985,8 +6991,9 @@ inline std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj)
 }
 
 typedef struct _CommitTxnRequest__isset {
-  _CommitTxnRequest__isset() : replPolicy(false) {}
+  _CommitTxnRequest__isset() : replPolicy(false), writeEventInfos(false) {}
   bool replPolicy :1;
+  bool writeEventInfos :1;
 } _CommitTxnRequest__isset;
 
 class CommitTxnRequest {
@@ -7000,6 +7007,7 @@ class CommitTxnRequest {
   virtual ~CommitTxnRequest() throw();
   int64_t txnid;
   std::string replPolicy;
+  std::vector<WriteEventInfo>  writeEventInfos;
 
   _CommitTxnRequest__isset __isset;
 
@@ -7007,6 +7015,8 @@ class CommitTxnRequest {
 
   void __set_replPolicy(const std::string& val);
 
+  void __set_writeEventInfos(const std::vector<WriteEventInfo> & val);
+
   bool operator == (const CommitTxnRequest & rhs) const
   {
     if (!(txnid == rhs.txnid))
@@ -7015,6 +7025,10 @@ class CommitTxnRequest {
       return false;
     else if (__isset.replPolicy && !(replPolicy == rhs.replPolicy))
       return false;
+    if (__isset.writeEventInfos != rhs.__isset.writeEventInfos)
+      return false;
+    else if (__isset.writeEventInfos && !(writeEventInfos == rhs.writeEventInfos))
+      return false;
     return true;
   }
   bool operator != (const CommitTxnRequest &rhs) const {
@@ -7037,6 +7051,90 @@ inline std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj)
   return out;
 }
 
+typedef struct _WriteEventInfo__isset {
+  _WriteEventInfo__isset() : partition(false), tableObj(false), partitionObj(false) {}
+  bool partition :1;
+  bool tableObj :1;
+  bool partitionObj :1;
+} _WriteEventInfo__isset;
+
+class WriteEventInfo {
+ public:
+
+  WriteEventInfo(const WriteEventInfo&);
+  WriteEventInfo& operator=(const WriteEventInfo&);
+  WriteEventInfo() : writeId(0), database(), table(), files(), partition(), tableObj(), partitionObj() {
+  }
+
+  virtual ~WriteEventInfo() throw();
+  int64_t writeId;
+  std::string database;
+  std::string table;
+  std::string files;
+  std::string partition;
+  std::string tableObj;
+  std::string partitionObj;
+
+  _WriteEventInfo__isset __isset;
+
+  void __set_writeId(const int64_t val);
+
+  void __set_database(const std::string& val);
+
+  void __set_table(const std::string& val);
+
+  void __set_files(const std::string& val);
+
+  void __set_partition(const std::string& val);
+
+  void __set_tableObj(const std::string& val);
+
+  void __set_partitionObj(const std::string& val);
+
+  bool operator == (const WriteEventInfo & rhs) const
+  {
+    if (!(writeId == rhs.writeId))
+      return false;
+    if (!(database == rhs.database))
+      return false;
+    if (!(table == rhs.table))
+      return false;
+    if (!(files == rhs.files))
+      return false;
+    if (__isset.partition != rhs.__isset.partition)
+      return false;
+    else if (__isset.partition && !(partition == rhs.partition))
+      return false;
+    if (__isset.tableObj != rhs.__isset.tableObj)
+      return false;
+    else if (__isset.tableObj && !(tableObj == rhs.tableObj))
+      return false;
+    if (__isset.partitionObj != rhs.__isset.partitionObj)
+      return false;
+    else if (__isset.partitionObj && !(partitionObj == rhs.partitionObj))
+      return false;
+    return true;
+  }
+  bool operator != (const WriteEventInfo &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteEventInfo & ) 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(WriteEventInfo &a, WriteEventInfo &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteEventInfo& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _ReplTblWriteIdStateRequest__isset {
   _ReplTblWriteIdStateRequest__isset() : partNames(false) {}
   bool partNames :1;
@@ -9035,9 +9133,10 @@ inline std::ostream& operator<<(std::ostream& out, const NotificationEventsCount
 }
 
 typedef struct _InsertEventRequestData__isset {
-  _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false) {}
+  _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false), subDirectoryList(false) {}
   bool replace :1;
   bool filesAddedChecksum :1;
+  bool subDirectoryList :1;
 } _InsertEventRequestData__isset;
 
 class InsertEventRequestData {
@@ -9052,6 +9151,7 @@ class InsertEventRequestData {
   bool replace;
   std::vector<std::string>  filesAdded;
   std::vector<std::string>  filesAddedChecksum;
+  std::vector<std::string>  subDirectoryList;
 
   _InsertEventRequestData__isset __isset;
 
@@ -9061,6 +9161,8 @@ class InsertEventRequestData {
 
   void __set_filesAddedChecksum(const std::vector<std::string> & val);
 
+  void __set_subDirectoryList(const std::vector<std::string> & val);
+
   bool operator == (const InsertEventRequestData & rhs) const
   {
     if (__isset.replace != rhs.__isset.replace)
@@ -9073,6 +9175,10 @@ class InsertEventRequestData {
       return false;
     else if (__isset.filesAddedChecksum && !(filesAddedChecksum == rhs.filesAddedChecksum))
       return false;
+    if (__isset.subDirectoryList != rhs.__isset.subDirectoryList)
+      return false;
+    else if (__isset.subDirectoryList && !(subDirectoryList == rhs.subDirectoryList))
+      return false;
     return true;
   }
   bool operator != (const InsertEventRequestData &rhs) const {
@@ -9258,6 +9364,114 @@ inline std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj)
   return out;
 }
 
+typedef struct _WriteNotificationLogRequest__isset {
+  _WriteNotificationLogRequest__isset() : partitionVals(false) {}
+  bool partitionVals :1;
+} _WriteNotificationLogRequest__isset;
+
+class WriteNotificationLogRequest {
+ public:
+
+  WriteNotificationLogRequest(const WriteNotificationLogRequest&);
+  WriteNotificationLogRequest& operator=(const WriteNotificationLogRequest&);
+  WriteNotificationLogRequest() : txnId(0), writeId(0), db(), table() {
+  }
+
+  virtual ~WriteNotificationLogRequest() throw();
+  int64_t txnId;
+  int64_t writeId;
+  std::string db;
+  std::string table;
+  InsertEventRequestData fileInfo;
+  std::vector<std::string>  partitionVals;
+
+  _WriteNotificationLogRequest__isset __isset;
+
+  void __set_txnId(const int64_t val);
+
+  void __set_writeId(const int64_t val);
+
+  void __set_db(const std::string& val);
+
+  void __set_table(const std::string& val);
+
+  void __set_fileInfo(const InsertEventRequestData& val);
+
+  void __set_partitionVals(const std::vector<std::string> & val);
+
+  bool operator == (const WriteNotificationLogRequest & rhs) const
+  {
+    if (!(txnId == rhs.txnId))
+      return false;
+    if (!(writeId == rhs.writeId))
+      return false;
+    if (!(db == rhs.db))
+      return false;
+    if (!(table == rhs.table))
+      return false;
+    if (!(fileInfo == rhs.fileInfo))
+      return false;
+    if (__isset.partitionVals != rhs.__isset.partitionVals)
+      return false;
+    else if (__isset.partitionVals && !(partitionVals == rhs.partitionVals))
+      return false;
+    return true;
+  }
+  bool operator != (const WriteNotificationLogRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteNotificationLogRequest & ) 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(WriteNotificationLogRequest &a, WriteNotificationLogRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteNotificationLogRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class WriteNotificationLogResponse {
+ public:
+
+  WriteNotificationLogResponse(const WriteNotificationLogResponse&);
+  WriteNotificationLogResponse& operator=(const WriteNotificationLogResponse&);
+  WriteNotificationLogResponse() {
+  }
+
+  virtual ~WriteNotificationLogResponse() throw();
+
+  bool operator == (const WriteNotificationLogResponse & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const WriteNotificationLogResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteNotificationLogResponse & ) 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(WriteNotificationLogResponse &a, WriteNotificationLogResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteNotificationLogResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _MetadataPpdResult__isset {
   _MetadataPpdResult__isset() : metadata(false), includeBitset(false) {}
   bool metadata :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 1dcc870..3ce72e9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list700.size);
-                String _elem701;
-                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list708.size);
+                String _elem709;
+                for (int _i710 = 0; _i710 < _list708.size; ++_i710)
                 {
-                  _elem701 = iprot.readString();
-                  struct.partitionnames.add(_elem701);
+                  _elem709 = iprot.readString();
+                  struct.partitionnames.add(_elem709);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter703 : struct.partitionnames)
+          for (String _iter711 : struct.partitionnames)
           {
-            oprot.writeString(_iter703);
+            oprot.writeString(_iter711);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter704 : struct.partitionnames)
+        for (String _iter712 : struct.partitionnames)
         {
-          oprot.writeString(_iter704);
+          oprot.writeString(_iter712);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list705.size);
-        String _elem706;
-        for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+        org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list713.size);
+        String _elem714;
+        for (int _i715 = 0; _i715 < _list713.size; ++_i715)
         {
-          _elem706 = iprot.readString();
-          struct.partitionnames.add(_elem706);
+          _elem714 = iprot.readString();
+          struct.partitionnames.add(_elem714);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index fa33963..a0b47a9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -716,13 +716,13 @@ import org.slf4j.LoggerFactory;
           case 3: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
-                struct.txnIds = new ArrayList<Long>(_list626.size);
-                long _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.txnIds = new ArrayList<Long>(_list634.size);
+                long _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem627 = iprot.readI64();
-                  struct.txnIds.add(_elem627);
+                  _elem635 = iprot.readI64();
+                  struct.txnIds.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -742,14 +742,14 @@ import org.slf4j.LoggerFactory;
           case 5: // SRC_TXN_TO_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list629 = iprot.readListBegin();
-                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list629.size);
-                TxnToWriteId _elem630;
-                for (int _i631 = 0; _i631 < _list629.size; ++_i631)
+                org.apache.thrift.protocol.TList _list637 = iprot.readListBegin();
+                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list637.size);
+                TxnToWriteId _elem638;
+                for (int _i639 = 0; _i639 < _list637.size; ++_i639)
                 {
-                  _elem630 = new TxnToWriteId();
-                  _elem630.read(iprot);
-                  struct.srcTxnToWriteIdList.add(_elem630);
+                  _elem638 = new TxnToWriteId();
+                  _elem638.read(iprot);
+                  struct.srcTxnToWriteIdList.add(_elem638);
                 }
                 iprot.readListEnd();
               }
@@ -786,9 +786,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-            for (long _iter632 : struct.txnIds)
+            for (long _iter640 : struct.txnIds)
             {
-              oprot.writeI64(_iter632);
+              oprot.writeI64(_iter640);
             }
             oprot.writeListEnd();
           }
@@ -807,9 +807,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SRC_TXN_TO_WRITE_ID_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.srcTxnToWriteIdList.size()));
-            for (TxnToWriteId _iter633 : struct.srcTxnToWriteIdList)
+            for (TxnToWriteId _iter641 : struct.srcTxnToWriteIdList)
             {
-              _iter633.write(oprot);
+              _iter641.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -849,9 +849,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnIds()) {
         {
           oprot.writeI32(struct.txnIds.size());
-          for (long _iter634 : struct.txnIds)
+          for (long _iter642 : struct.txnIds)
           {
-            oprot.writeI64(_iter634);
+            oprot.writeI64(_iter642);
           }
         }
       }
@@ -861,9 +861,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSrcTxnToWriteIdList()) {
         {
           oprot.writeI32(struct.srcTxnToWriteIdList.size());
-          for (TxnToWriteId _iter635 : struct.srcTxnToWriteIdList)
+          for (TxnToWriteId _iter643 : struct.srcTxnToWriteIdList)
           {
-            _iter635.write(oprot);
+            _iter643.write(oprot);
           }
         }
       }
@@ -879,13 +879,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list636 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.txnIds = new ArrayList<Long>(_list636.size);
-          long _elem637;
-          for (int _i638 = 0; _i638 < _list636.size; ++_i638)
+          org.apache.thrift.protocol.TList _list644 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.txnIds = new ArrayList<Long>(_list644.size);
+          long _elem645;
+          for (int _i646 = 0; _i646 < _list644.size; ++_i646)
           {
-            _elem637 = iprot.readI64();
-            struct.txnIds.add(_elem637);
+            _elem645 = iprot.readI64();
+            struct.txnIds.add(_elem645);
           }
         }
         struct.setTxnIdsIsSet(true);
@@ -896,14 +896,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list639.size);
-          TxnToWriteId _elem640;
-          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list647.size);
+          TxnToWriteId _elem648;
+          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
           {
-            _elem640 = new TxnToWriteId();
-            _elem640.read(iprot);
-            struct.srcTxnToWriteIdList.add(_elem640);
+            _elem648 = new TxnToWriteId();
+            _elem648.read(iprot);
+            struct.srcTxnToWriteIdList.add(_elem648);
           }
         }
         struct.setSrcTxnToWriteIdListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index 20dc757..13df26d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list642.size);
-                TxnToWriteId _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list650.size);
+                TxnToWriteId _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem643 = new TxnToWriteId();
-                  _elem643.read(iprot);
-                  struct.txnToWriteIds.add(_elem643);
+                  _elem651 = new TxnToWriteId();
+                  _elem651.read(iprot);
+                  struct.txnToWriteIds.add(_elem651);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter645 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter653 : struct.txnToWriteIds)
           {
-            _iter645.write(oprot);
+            _iter653.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter646 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter654 : struct.txnToWriteIds)
         {
-          _iter646.write(oprot);
+          _iter654.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list647.size);
-        TxnToWriteId _elem648;
-        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list655.size);
+        TxnToWriteId _elem656;
+        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
         {
-          _elem648 = new TxnToWriteId();
-          _elem648.read(iprot);
-          struct.txnToWriteIds.add(_elem648);
+          _elem656 = new TxnToWriteId();
+          _elem656.read(iprot);
+          struct.txnToWriteIds.add(_elem656);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index 470a070..1af1628 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list800.size);
-                long _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list824.size);
+                long _elem825;
+                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
                 {
-                  _elem801 = iprot.readI64();
-                  struct.fileIds.add(_elem801);
+                  _elem825 = iprot.readI64();
+                  struct.fileIds.add(_elem825);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter803 : struct.fileIds)
+          for (long _iter827 : struct.fileIds)
           {
-            oprot.writeI64(_iter803);
+            oprot.writeI64(_iter827);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter804 : struct.fileIds)
+        for (long _iter828 : struct.fileIds)
         {
-          oprot.writeI64(_iter804);
+          oprot.writeI64(_iter828);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list805.size);
-        long _elem806;
-        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+        org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list829.size);
+        long _elem830;
+        for (int _i831 = 0; _i831 < _list829.size; ++_i831)
         {
-          _elem806 = iprot.readI64();
-          struct.fileIds.add(_elem806);
+          _elem830 = iprot.readI64();
+          struct.fileIds.add(_elem830);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index af48583..4cd04f1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list816 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list816.size);
-                ClientCapability _elem817;
-                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
+                org.apache.thrift.protocol.TList _list840 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list840.size);
+                ClientCapability _elem841;
+                for (int _i842 = 0; _i842 < _list840.size; ++_i842)
                 {
-                  _elem817 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem817);
+                  _elem841 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem841);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter819 : struct.values)
+          for (ClientCapability _iter843 : struct.values)
           {
-            oprot.writeI32(_iter819.getValue());
+            oprot.writeI32(_iter843.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter820 : struct.values)
+        for (ClientCapability _iter844 : struct.values)
         {
-          oprot.writeI32(_iter820.getValue());
+          oprot.writeI32(_iter844.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list821.size);
-        ClientCapability _elem822;
-        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
+        org.apache.thrift.protocol.TList _list845 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list845.size);
+        ClientCapability _elem846;
+        for (int _i847 = 0; _i847 < _list845.size; ++_i847)
         {
-          _elem822 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem822);
+          _elem846 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem846);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
index 3c15f84..f295958 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
   private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1);
   private static final org.apache.thrift.protocol.TField REPL_POLICY_FIELD_DESC = new org.apache.thrift.protocol.TField("replPolicy", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField WRITE_EVENT_INFOS_FIELD_DESC = new org.apache.thrift.protocol.TField("writeEventInfos", org.apache.thrift.protocol.TType.LIST, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,13 @@ import org.slf4j.LoggerFactory;
 
   private long txnid; // required
   private String replPolicy; // optional
+  private List<WriteEventInfo> writeEventInfos; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     TXNID((short)1, "txnid"),
-    REPL_POLICY((short)2, "replPolicy");
+    REPL_POLICY((short)2, "replPolicy"),
+    WRITE_EVENT_INFOS((short)3, "writeEventInfos");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +75,8 @@ import org.slf4j.LoggerFactory;
           return TXNID;
         case 2: // REPL_POLICY
           return REPL_POLICY;
+        case 3: // WRITE_EVENT_INFOS
+          return WRITE_EVENT_INFOS;
         default:
           return null;
       }
@@ -114,7 +119,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __TXNID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.REPL_POLICY};
+  private static final _Fields optionals[] = {_Fields.REPL_POLICY,_Fields.WRITE_EVENT_INFOS};
   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);
@@ -122,6 +127,9 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.REPL_POLICY, new org.apache.thrift.meta_data.FieldMetaData("replPolicy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.WRITE_EVENT_INFOS, new org.apache.thrift.meta_data.FieldMetaData("writeEventInfos", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "WriteEventInfo"))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommitTxnRequest.class, metaDataMap);
   }
@@ -146,6 +154,13 @@ import org.slf4j.LoggerFactory;
     if (other.isSetReplPolicy()) {
       this.replPolicy = other.replPolicy;
     }
+    if (other.isSetWriteEventInfos()) {
+      List<WriteEventInfo> __this__writeEventInfos = new ArrayList<WriteEventInfo>(other.writeEventInfos.size());
+      for (WriteEventInfo other_element : other.writeEventInfos) {
+        __this__writeEventInfos.add(other_element);
+      }
+      this.writeEventInfos = __this__writeEventInfos;
+    }
   }
 
   public CommitTxnRequest deepCopy() {
@@ -157,6 +172,7 @@ import org.slf4j.LoggerFactory;
     setTxnidIsSet(false);
     this.txnid = 0;
     this.replPolicy = null;
+    this.writeEventInfos = null;
   }
 
   public long getTxnid() {
@@ -204,6 +220,44 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getWriteEventInfosSize() {
+    return (this.writeEventInfos == null) ? 0 : this.writeEventInfos.size();
+  }
+
+  public java.util.Iterator<WriteEventInfo> getWriteEventInfosIterator() {
+    return (this.writeEventInfos == null) ? null : this.writeEventInfos.iterator();
+  }
+
+  public void addToWriteEventInfos(WriteEventInfo elem) {
+    if (this.writeEventInfos == null) {
+      this.writeEventInfos = new ArrayList<WriteEventInfo>();
+    }
+    this.writeEventInfos.add(elem);
+  }
+
+  public List<WriteEventInfo> getWriteEventInfos() {
+    return this.writeEventInfos;
+  }
+
+  public void setWriteEventInfos(List<WriteEventInfo> writeEventInfos) {
+    this.writeEventInfos = writeEventInfos;
+  }
+
+  public void unsetWriteEventInfos() {
+    this.writeEventInfos = null;
+  }
+
+  /** Returns true if field writeEventInfos is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteEventInfos() {
+    return this.writeEventInfos != null;
+  }
+
+  public void setWriteEventInfosIsSet(boolean value) {
+    if (!value) {
+      this.writeEventInfos = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TXNID:
@@ -222,6 +276,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WRITE_EVENT_INFOS:
+      if (value == null) {
+        unsetWriteEventInfos();
+      } else {
+        setWriteEventInfos((List<WriteEventInfo>)value);
+      }
+      break;
+
     }
   }
 
@@ -233,6 +295,9 @@ import org.slf4j.LoggerFactory;
     case REPL_POLICY:
       return getReplPolicy();
 
+    case WRITE_EVENT_INFOS:
+      return getWriteEventInfos();
+
     }
     throw new IllegalStateException();
   }
@@ -248,6 +313,8 @@ import org.slf4j.LoggerFactory;
       return isSetTxnid();
     case REPL_POLICY:
       return isSetReplPolicy();
+    case WRITE_EVENT_INFOS:
+      return isSetWriteEventInfos();
     }
     throw new IllegalStateException();
   }
@@ -283,6 +350,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_writeEventInfos = true && this.isSetWriteEventInfos();
+    boolean that_present_writeEventInfos = true && that.isSetWriteEventInfos();
+    if (this_present_writeEventInfos || that_present_writeEventInfos) {
+      if (!(this_present_writeEventInfos && that_present_writeEventInfos))
+        return false;
+      if (!this.writeEventInfos.equals(that.writeEventInfos))
+        return false;
+    }
+
     return true;
   }
 
@@ -300,6 +376,11 @@ import org.slf4j.LoggerFactory;
     if (present_replPolicy)
       list.add(replPolicy);
 
+    boolean present_writeEventInfos = true && (isSetWriteEventInfos());
+    list.add(present_writeEventInfos);
+    if (present_writeEventInfos)
+      list.add(writeEventInfos);
+
     return list.hashCode();
   }
 
@@ -331,6 +412,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetWriteEventInfos()).compareTo(other.isSetWriteEventInfos());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteEventInfos()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeEventInfos, other.writeEventInfos);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -364,6 +455,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetWriteEventInfos()) {
+      if (!first) sb.append(", ");
+      sb.append("writeEventInfos:");
+      if (this.writeEventInfos == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.writeEventInfos);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -429,6 +530,25 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // WRITE_EVENT_INFOS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
+                struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list594.size);
+                WriteEventInfo _elem595;
+                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                {
+                  _elem595 = new WriteEventInfo();
+                  _elem595.read(iprot);
+                  struct.writeEventInfos.add(_elem595);
+                }
+                iprot.readListEnd();
+              }
+              struct.setWriteEventInfosIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -452,6 +572,20 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.writeEventInfos != null) {
+        if (struct.isSetWriteEventInfos()) {
+          oprot.writeFieldBegin(WRITE_EVENT_INFOS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.writeEventInfos.size()));
+            for (WriteEventInfo _iter597 : struct.writeEventInfos)
+            {
+              _iter597.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -474,10 +608,22 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetReplPolicy()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetWriteEventInfos()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetReplPolicy()) {
         oprot.writeString(struct.replPolicy);
       }
+      if (struct.isSetWriteEventInfos()) {
+        {
+          oprot.writeI32(struct.writeEventInfos.size());
+          for (WriteEventInfo _iter598 : struct.writeEventInfos)
+          {
+            _iter598.write(oprot);
+          }
+        }
+      }
     }
 
     @Override
@@ -485,11 +631,25 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.txnid = iprot.readI64();
       struct.setTxnidIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.replPolicy = iprot.readString();
         struct.setReplPolicyIsSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list599.size);
+          WriteEventInfo _elem600;
+          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+          {
+            _elem600 = new WriteEventInfo();
+            _elem600.read(iprot);
+            struct.writeEventInfos.add(_elem600);
+          }
+        }
+        struct.setWriteEventInfosIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 31f2e14..57eb5ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map682 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map682.size);
-                String _key683;
-                String _val684;
-                for (int _i685 = 0; _i685 < _map682.size; ++_i685)
+                org.apache.thrift.protocol.TMap _map690 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map690.size);
+                String _key691;
+                String _val692;
+                for (int _i693 = 0; _i693 < _map690.size; ++_i693)
                 {
-                  _key683 = iprot.readString();
-                  _val684 = iprot.readString();
-                  struct.properties.put(_key683, _val684);
+                  _key691 = iprot.readString();
+                  _val692 = iprot.readString();
+                  struct.properties.put(_key691, _val692);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter686 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter694 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter686.getKey());
-              oprot.writeString(_iter686.getValue());
+              oprot.writeString(_iter694.getKey());
+              oprot.writeString(_iter694.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter687 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter695 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter687.getKey());
-            oprot.writeString(_iter687.getValue());
+            oprot.writeString(_iter695.getKey());
+            oprot.writeString(_iter695.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map688 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map688.size);
-          String _key689;
-          String _val690;
-          for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+          org.apache.thrift.protocol.TMap _map696 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map696.size);
+          String _key697;
+          String _val698;
+          for (int _i699 = 0; _i699 < _map696.size; ++_i699)
           {
-            _key689 = iprot.readString();
-            _val690 = iprot.readString();
-            struct.properties.put(_key689, _val690);
+            _key697 = iprot.readString();
+            _val698 = iprot.readString();
+            struct.properties.put(_key697, _val698);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index ab7b059..611bf6f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -712,13 +712,13 @@ import org.slf4j.LoggerFactory;
           case 4: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set708 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set708.size);
-                String _elem709;
-                for (int _i710 = 0; _i710 < _set708.size; ++_i710)
+                org.apache.thrift.protocol.TSet _set716 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set716.size);
+                String _elem717;
+                for (int _i718 = 0; _i718 < _set716.size; ++_i718)
                 {
-                  _elem709 = iprot.readString();
-                  struct.tablesUsed.add(_elem709);
+                  _elem717 = iprot.readString();
+                  struct.tablesUsed.add(_elem717);
                 }
                 iprot.readSetEnd();
               }
@@ -767,9 +767,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter711 : struct.tablesUsed)
+          for (String _iter719 : struct.tablesUsed)
           {
-            oprot.writeString(_iter711);
+            oprot.writeString(_iter719);
           }
           oprot.writeSetEnd();
         }
@@ -804,9 +804,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter712 : struct.tablesUsed)
+        for (String _iter720 : struct.tablesUsed)
         {
-          oprot.writeString(_iter712);
+          oprot.writeString(_iter720);
         }
       }
       BitSet optionals = new BitSet();
@@ -829,13 +829,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set713 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set713.size);
-        String _elem714;
-        for (int _i715 = 0; _i715 < _set713.size; ++_i715)
+        org.apache.thrift.protocol.TSet _set721 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set721.size);
+        String _elem722;
+        for (int _i723 = 0; _i723 < _set721.size; ++_i723)
         {
-          _elem714 = iprot.readString();
-          struct.tablesUsed.add(_elem714);
+          _elem722 = iprot.readString();
+          struct.tablesUsed.add(_elem722);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index e43493e..8f5b4e5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list920.size);
-                SchemaVersionDescriptor _elem921;
-                for (int _i922 = 0; _i922 < _list920.size; ++_i922)
+                org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list944.size);
+                SchemaVersionDescriptor _elem945;
+                for (int _i946 = 0; _i946 < _list944.size; ++_i946)
                 {
-                  _elem921 = new SchemaVersionDescriptor();
-                  _elem921.read(iprot);
-                  struct.schemaVersions.add(_elem921);
+                  _elem945 = new SchemaVersionDescriptor();
+                  _elem945.read(iprot);
+                  struct.schemaVersions.add(_elem945);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter923 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter947 : struct.schemaVersions)
           {
-            _iter923.write(oprot);
+            _iter947.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter924 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter948 : struct.schemaVersions)
           {
-            _iter924.write(oprot);
+            _iter948.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list925.size);
-          SchemaVersionDescriptor _elem926;
-          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+          org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list949.size);
+          SchemaVersionDescriptor _elem950;
+          for (int _i951 = 0; _i951 < _list949.size; ++_i951)
           {
-            _elem926 = new SchemaVersionDescriptor();
-            _elem926.read(iprot);
-            struct.schemaVersions.add(_elem926);
+            _elem950 = new SchemaVersionDescriptor();
+            _elem950.read(iprot);
+            struct.schemaVersions.add(_elem950);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 7b0ec6c..2560922 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -794,13 +794,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list740.size);
-                String _elem741;
-                for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list756.size);
+                String _elem757;
+                for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                 {
-                  _elem741 = iprot.readString();
-                  struct.partitionVals.add(_elem741);
+                  _elem757 = iprot.readString();
+                  struct.partitionVals.add(_elem757);
                 }
                 iprot.readListEnd();
               }
@@ -857,9 +857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter743 : struct.partitionVals)
+            for (String _iter759 : struct.partitionVals)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -915,9 +915,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter744 : struct.partitionVals)
+          for (String _iter760 : struct.partitionVals)
           {
-            oprot.writeString(_iter744);
+            oprot.writeString(_iter760);
           }
         }
       }
@@ -945,13 +945,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list745.size);
-          String _elem746;
-          for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+          org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list761.size);
+          String _elem762;
+          for (int _i763 = 0; _i763 < _list761.size; ++_i763)
           {
-            _elem746 = iprot.readString();
-            struct.partitionVals.add(_elem746);
+            _elem762 = iprot.readString();
+            struct.partitionVals.add(_elem762);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 544ba19..f68afe8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list808.size);
-                Function _elem809;
-                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
+                org.apache.thrift.protocol.TList _list832 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list832.size);
+                Function _elem833;
+                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
                 {
-                  _elem809 = new Function();
-                  _elem809.read(iprot);
-                  struct.functions.add(_elem809);
+                  _elem833 = new Function();
+                  _elem833.read(iprot);
+                  struct.functions.add(_elem833);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter811 : struct.functions)
+            for (Function _iter835 : struct.functions)
             {
-              _iter811.write(oprot);
+              _iter835.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter812 : struct.functions)
+          for (Function _iter836 : struct.functions)
           {
-            _iter812.write(oprot);
+            _iter836.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list813 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list813.size);
-          Function _elem814;
-          for (int _i815 = 0; _i815 < _list813.size; ++_i815)
+          org.apache.thrift.protocol.TList _list837 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list837.size);
+          Function _elem838;
+          for (int _i839 = 0; _i839 < _list837.size; ++_i839)
           {
-            _elem814 = new Function();
-            _elem814.read(iprot);
-            struct.functions.add(_elem814);
+            _elem838 = new Function();
+            _elem838.read(iprot);
+            struct.functions.add(_elem838);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 0a94f2f..836f35f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list758 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list758.size);
-                long _elem759;
-                for (int _i760 = 0; _i760 < _list758.size; ++_i760)
+                org.apache.thrift.protocol.TList _list782 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list782.size);
+                long _elem783;
+                for (int _i784 = 0; _i784 < _list782.size; ++_i784)
                 {
-                  _elem759 = iprot.readI64();
-                  struct.fileIds.add(_elem759);
+                  _elem783 = iprot.readI64();
+                  struct.fileIds.add(_elem783);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter761 : struct.fileIds)
+          for (long _iter785 : struct.fileIds)
           {
-            oprot.writeI64(_iter761);
+            oprot.writeI64(_iter785);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter762 : struct.fileIds)
+        for (long _iter786 : struct.fileIds)
         {
-          oprot.writeI64(_iter762);
+          oprot.writeI64(_iter786);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list763 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list763.size);
-        long _elem764;
-        for (int _i765 = 0; _i765 < _list763.size; ++_i765)
+        org.apache.thrift.protocol.TList _list787 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list787.size);
+        long _elem788;
+        for (int _i789 = 0; _i789 < _list787.size; ++_i789)
         {
-          _elem764 = iprot.readI64();
-          struct.fileIds.add(_elem764);
+          _elem788 = iprot.readI64();
+          struct.fileIds.add(_elem788);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index e07d2e5..17f0ee5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map748 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map748.size);
-                long _key749;
-                MetadataPpdResult _val750;
-                for (int _i751 = 0; _i751 < _map748.size; ++_i751)
+                org.apache.thrift.protocol.TMap _map772 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map772.size);
+                long _key773;
+                MetadataPpdResult _val774;
+                for (int _i775 = 0; _i775 < _map772.size; ++_i775)
                 {
-                  _key749 = iprot.readI64();
-                  _val750 = new MetadataPpdResult();
-                  _val750.read(iprot);
-                  struct.metadata.put(_key749, _val750);
+                  _key773 = iprot.readI64();
+                  _val774 = new MetadataPpdResult();
+                  _val774.read(iprot);
+                  struct.metadata.put(_key773, _val774);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter752 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter776 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter752.getKey());
-            _iter752.getValue().write(oprot);
+            oprot.writeI64(_iter776.getKey());
+            _iter776.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter753 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter777 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter753.getKey());
-          _iter753.getValue().write(oprot);
+          oprot.writeI64(_iter777.getKey());
+          _iter777.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map754 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map754.size);
-        long _key755;
-        MetadataPpdResult _val756;
-        for (int _i757 = 0; _i757 < _map754.size; ++_i757)
+        org.apache.thrift.protocol.TMap _map778 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map778.size);
+        long _key779;
+        MetadataPpdResult _val780;
+        for (int _i781 = 0; _i781 < _map778.size; ++_i781)
         {
-          _key755 = iprot.readI64();
-          _val756 = new MetadataPpdResult();
-          _val756.read(iprot);
-          struct.metadata.put(_key755, _val756);
+          _key779 = iprot.readI64();
+          _val780 = new MetadataPpdResult();
+          _val780.read(iprot);
+          struct.metadata.put(_key779, _val780);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index ebb6639..12b4392 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list776 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list776.size);
-                long _elem777;
-                for (int _i778 = 0; _i778 < _list776.size; ++_i778)
+                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list800.size);
+                long _elem801;
+                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
                 {
-                  _elem777 = iprot.readI64();
-                  struct.fileIds.add(_elem777);
+                  _elem801 = iprot.readI64();
+                  struct.fileIds.add(_elem801);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter779 : struct.fileIds)
+          for (long _iter803 : struct.fileIds)
           {
-            oprot.writeI64(_iter779);
+            oprot.writeI64(_iter803);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter780 : struct.fileIds)
+        for (long _iter804 : struct.fileIds)
         {
-          oprot.writeI64(_iter780);
+          oprot.writeI64(_iter804);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list781 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list781.size);
-        long _elem782;
-        for (int _i783 = 0; _i783 < _list781.size; ++_i783)
+        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list805.size);
+        long _elem806;
+        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
         {
-          _elem782 = iprot.readI64();
-          struct.fileIds.add(_elem782);
+          _elem806 = iprot.readI64();
+          struct.fileIds.add(_elem806);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 67981cd..65708d7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map766 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map766.size);
-                long _key767;
-                ByteBuffer _val768;
-                for (int _i769 = 0; _i769 < _map766.size; ++_i769)
+                org.apache.thrift.protocol.TMap _map790 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map790.size);
+                long _key791;
+                ByteBuffer _val792;
+                for (int _i793 = 0; _i793 < _map790.size; ++_i793)
                 {
-                  _key767 = iprot.readI64();
-                  _val768 = iprot.readBinary();
-                  struct.metadata.put(_key767, _val768);
+                  _key791 = iprot.readI64();
+                  _val792 = iprot.readBinary();
+                  struct.metadata.put(_key791, _val792);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter770 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter794 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter770.getKey());
-            oprot.writeBinary(_iter770.getValue());
+            oprot.writeI64(_iter794.getKey());
+            oprot.writeBinary(_iter794.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter771 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter795 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter771.getKey());
-          oprot.writeBinary(_iter771.getValue());
+          oprot.writeI64(_iter795.getKey());
+          oprot.writeBinary(_iter795.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map772 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map772.size);
-        long _key773;
-        ByteBuffer _val774;
-        for (int _i775 = 0; _i775 < _map772.size; ++_i775)
+        org.apache.thrift.protocol.TMap _map796 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map796.size);
+        long _key797;
+        ByteBuffer _val798;
+        for (int _i799 = 0; _i799 < _map796.size; ++_i799)
         {
-          _key773 = iprot.readI64();
-          _val774 = iprot.readBinary();
-          struct.metadata.put(_key773, _val774);
+          _key797 = iprot.readI64();
+          _val798 = iprot.readBinary();
+          struct.metadata.put(_key797, _val798);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 6a78b77..09ca865 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -606,13 +606,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list824.size);
-                String _elem825;
-                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
+                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list848.size);
+                String _elem849;
+                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
                 {
-                  _elem825 = iprot.readString();
-                  struct.tblNames.add(_elem825);
+                  _elem849 = iprot.readString();
+                  struct.tblNames.add(_elem849);
                 }
                 iprot.readListEnd();
               }
@@ -661,9 +661,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter827 : struct.tblNames)
+            for (String _iter851 : struct.tblNames)
             {
-              oprot.writeString(_iter827);
+              oprot.writeString(_iter851);
             }
             oprot.writeListEnd();
           }
@@ -716,9 +716,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter828 : struct.tblNames)
+          for (String _iter852 : struct.tblNames)
           {
-            oprot.writeString(_iter828);
+            oprot.writeString(_iter852);
           }
         }
       }
@@ -738,13 +738,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list829.size);
-          String _elem830;
-          for (int _i831 = 0; _i831 < _list829.size; ++_i831)
+          org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list853.size);
+          String _elem854;
+          for (int _i855 = 0; _i855 < _list853.size; ++_i855)
           {
-            _elem830 = iprot.readString();
-            struct.tblNames.add(_elem830);
+            _elem854 = iprot.readString();
+            struct.tblNames.add(_elem854);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 13be2ed..72256e6 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list832 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list832.size);
-                Table _elem833;
-                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
+                org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list856.size);
+                Table _elem857;
+                for (int _i858 = 0; _i858 < _list856.size; ++_i858)
                 {
-                  _elem833 = new Table();
-                  _elem833.read(iprot);
-                  struct.tables.add(_elem833);
+                  _elem857 = new Table();
+                  _elem857.read(iprot);
+                  struct.tables.add(_elem857);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter835 : struct.tables)
+          for (Table _iter859 : struct.tables)
           {
-            _iter835.write(oprot);
+            _iter859.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter836 : struct.tables)
+        for (Table _iter860 : struct.tables)
         {
-          _iter836.write(oprot);
+          _iter860.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list837 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list837.size);
-        Table _elem838;
-        for (int _i839 = 0; _i839 < _list837.size; ++_i839)
+        org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list861.size);
+        Table _elem862;
+        for (int _i863 = 0; _i863 < _list861.size; ++_i863)
         {
-          _elem838 = new Table();
-          _elem838.read(iprot);
-          struct.tables.add(_elem838);
+          _elem862 = new Table();
+          _elem862.read(iprot);
+          struct.tables.add(_elem862);
         }
       }
       struct.setTablesIsSet(true);


[15/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
index 9e0ce82..5e113c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
@@ -179,7 +179,7 @@ public class IncrementalLoadTasksBuilder {
       Database database;
       try {
         database = Hive.get().getDatabase(dbName);
-        return isEventNotReplayed(database.getParameters(), dir, dumpType);
+        return database == null ? true : isEventNotReplayed(database.getParameters(), dir, dumpType);
       } catch (HiveException e) {
         //may be the db is getting created in this load
         log.debug("failed to get the database " + dbName);
@@ -255,50 +255,55 @@ public class IncrementalLoadTasksBuilder {
     return updateReplIdTask;
   }
 
-  private List<Task<? extends Serializable>> addUpdateReplStateTasks(boolean isDatabaseLoad,
-                                           UpdatedMetaDataTracker updatedMetadata,
-                                           List<Task<? extends Serializable>> importTasks) throws SemanticException {
-    String replState = updatedMetadata.getReplicationState();
-    String database = updatedMetadata.getDatabase();
-    String table = updatedMetadata.getTable();
-
-    // If no import tasks generated by the event or no table updated for table level load, then no
-    // need to update the repl state to any object.
-    if (importTasks.isEmpty() || (!isDatabaseLoad && (table == null))) {
-      log.debug("No objects need update of repl state: Either 0 import tasks or table level load");
+  private List<Task<? extends Serializable>> addUpdateReplStateTasks(
+          boolean isDatabaseLoad,
+          UpdatedMetaDataTracker updatedMetaDataTracker,
+          List<Task<? extends Serializable>> importTasks) throws SemanticException {
+    // If no import tasks generated by the event then no need to update the repl state to any object.
+    if (importTasks.isEmpty()) {
+      log.debug("No objects need update of repl state: 0 import tasks");
       return importTasks;
     }
 
     // Create a barrier task for dependency collection of import tasks
-    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork());
-
-    // Link import tasks to the barrier task which will in-turn linked with repl state update tasks
-    for (Task<? extends Serializable> t : importTasks){
-      t.addDependentTask(barrierTask);
-      log.debug("Added {}:{} as a precursor of barrier task {}:{}",
-              t.getClass(), t.getId(), barrierTask.getClass(), barrierTask.getId());
-    }
-
+    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork(), conf);
     List<Task<? extends Serializable>> tasks = new ArrayList<>();
     Task<? extends Serializable> updateReplIdTask;
 
-    // If any partition is updated, then update repl state in partition object
-    for (final Map<String, String> partSpec : updatedMetadata.getPartitions()) {
-      updateReplIdTask = tableUpdateReplStateTask(database, table, partSpec, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    for (UpdatedMetaDataTracker.UpdateMetaData updateMetaData : updatedMetaDataTracker.getUpdateMetaDataList()) {
+      String replState = updateMetaData.getReplState();
+      String dbName = updateMetaData.getDbName();
+      String tableName = updateMetaData.getTableName();
+      // If any partition is updated, then update repl state in partition object
+      for (final Map<String, String> partSpec : updateMetaData.getPartitionsList()) {
+        updateReplIdTask = tableUpdateReplStateTask(dbName, tableName, partSpec, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
+
+      if (tableName != null) {
+        // If any table/partition is updated, then update repl state in table object
+        updateReplIdTask = tableUpdateReplStateTask(dbName, tableName, null, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
+
+      // For table level load, need not update replication state for the database
+      if (isDatabaseLoad) {
+        // If any table/partition is updated, then update repl state in db object
+        updateReplIdTask = dbUpdateReplStateTask(dbName, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
     }
 
-    if (table != null) {
-      // If any table/partition is updated, then update repl state in table object
-      updateReplIdTask = tableUpdateReplStateTask(database, table, null, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    if (tasks.isEmpty()) {
+      log.debug("No objects need update of repl state: 0 update tracker tasks");
+      return importTasks;
     }
 
-    // For table level load, need not update replication state for the database
-    if (isDatabaseLoad) {
-      // If any table/partition is updated, then update repl state in db object
-      updateReplIdTask = dbUpdateReplStateTask(database, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    // Link import tasks to the barrier task which will in-turn linked with repl state update tasks
+    for (Task<? extends Serializable> t : importTasks){
+      t.addDependentTask(barrierTask);
+      log.debug("Added {}:{} as a precursor of barrier task {}:{}",
+              t.getClass(), t.getId(), barrierTask.getClass(), barrierTask.getId());
     }
 
     // At least one task would have been added to update the repl state

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 7fce67f..16ba82e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hive.ql.exec.Utilities.COPY_KEYWORD;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -71,21 +70,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.io.Serializable;
 import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.apache.hadoop.hive.ql.exec.Utilities.COPY_KEYWORD;
-
 
 /**
  * Utilities that are shared by all of the ACID input and output formats. They
@@ -1907,6 +1892,28 @@ public class AcidUtils {
     return null;
   }
 
+  //Get the first level acid directory (if any) from a given path
+  public static String getFirstLevelAcidDirPath(Path dataPath, FileSystem fileSystem) throws IOException {
+    if (dataPath == null) {
+      return null;
+    }
+    String firstLevelAcidDir = getAcidSubDir(dataPath);
+    if (firstLevelAcidDir != null) {
+      return firstLevelAcidDir;
+    }
+
+    String acidDirPath = getFirstLevelAcidDirPath(dataPath.getParent(), fileSystem);
+    if (acidDirPath == null) {
+      return null;
+    }
+
+    // We need the path for directory so no need to append file name
+    if (fileSystem.isDirectory(dataPath)) {
+      return acidDirPath + Path.SEPARATOR + dataPath.getName();
+    }
+    return acidDirPath;
+  }
+
   public static boolean isAcidEnabled(HiveConf hiveConf) {
     String txnMgr = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER);
     boolean concurrency =  hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index bcc0508..ec8527e 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -26,30 +26,32 @@ import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.StringInternUtils;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hive.common.util.HiveStringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.JavaUtils;
-import org.apache.hadoop.hive.common.StringInternUtils;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.hive.llap.io.api.LlapIo;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner;
@@ -62,8 +64,6 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc;
-import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc.VectorMapOperatorReadType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.Deserializer;
@@ -78,10 +78,7 @@ import org.apache.hadoop.mapred.JobConfigurable;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hive.common.util.Ref;
 import org.apache.hive.common.util.ReflectionUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * HiveInputFormat is a parameterized InputFormat which looks at the path name
@@ -460,8 +457,9 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       InputFormat inputFormat, Class<? extends InputFormat> inputFormatClass, int splits,
       TableDesc table, List<InputSplit> result)
           throws IOException {
+    String tableName = table.getTableName();
     ValidWriteIdList validWriteIdList = AcidUtils.getTableValidWriteIdList(
-        conf, table.getTableName());
+        conf, tableName == null ? null : HiveStringUtils.normalizeIdentifier(tableName));
     ValidWriteIdList validMmWriteIdList = getMmValidWriteIds(conf, table, validWriteIdList);
 
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 4fd1d4e..78980fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Context;
@@ -638,14 +639,15 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public void replCommitTxn(String replPolicy, long srcTxnId) throws LockException {
+  public void replCommitTxn(CommitTxnRequest rqst) throws LockException {
     try {
-      getMS().replCommitTxn(srcTxnId, replPolicy);
+      getMS().replCommitTxn(rqst);
     } catch (NoSuchTxnException e) {
-      LOG.error("Metastore could not find " + JavaUtils.txnIdToString(srcTxnId));
-      throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(srcTxnId));
+      LOG.error("Metastore could not find " + JavaUtils.txnIdToString(rqst.getTxnid()));
+      throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(rqst.getTxnid()));
     } catch (TxnAbortedException e) {
-      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(srcTxnId), e.getMessage());
+      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED,
+              JavaUtils.txnIdToString(rqst.getTxnid()), e.getMessage());
       LOG.error(le.getMessage());
       throw le;
     } catch (TException e) {
@@ -1013,7 +1015,11 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     assert isTxnOpen();
     return stmtId++;
   }
-
+  @Override
+  public int getCurrentStmtId() {
+    assert isTxnOpen();
+    return stmtId;
+  }
   @Override
   public long getTableWriteId(String dbName, String tableName) throws LockException {
     assert isTxnOpen();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index ab9d67e..1feddeb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -74,6 +75,10 @@ class DummyTxnManager extends HiveTxnManagerImpl {
     return 0;
   }
   @Override
+  public int getCurrentStmtId() {
+    return  0;
+  }
+  @Override
   public long getTableWriteId(String dbName, String tableName) throws LockException {
     return 0L;
   }
@@ -220,7 +225,7 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public void replCommitTxn(String replPolicy, long srcTxnId) throws LockException {
+  public void replCommitTxn(CommitTxnRequest rqst) throws LockException {
     // No-op
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
index 5f68e08..9575552 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.Context;
@@ -61,11 +62,11 @@ public interface HiveTxnManager {
 
   /**
    * Commit the transaction in target cluster.
-   * @param replPolicy Replication policy to uniquely identify the source cluster.
-   * @param srcTxnId The id of the transaction at the source cluster
+   *
+   * @param rqst Commit transaction request having information related to commit txn and write events.
    * @throws LockException in case of failure to commit the transaction.
    */
-  void replCommitTxn(String replPolicy, long srcTxnId) throws LockException;
+  void replCommitTxn(CommitTxnRequest rqst) throws LockException;
 
  /**
    * Abort the transaction in target cluster.
@@ -295,6 +296,9 @@ public interface HiveTxnManager {
    */
   int getStmtIdAndIncrement();
 
+  // Can be used by operation to set the stmt id when allocation is done somewhere else.
+  int getCurrentStmtId();
+
   /**
    * Acquire the materialization rebuild lock for a given view. We need to specify the fully
    * qualified name of the materialized view and the open transaction ID so we can identify

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/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 c3809d8..c2ffe02 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
@@ -159,6 +159,7 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator;
@@ -1719,6 +1720,13 @@ public class Hive {
       List<Path> newFiles = Collections.synchronizedList(new ArrayList<Path>());
 
       perfLogger.PerfLogBegin("MoveTask", PerfLogger.FILE_MOVES);
+      
+      // If config is set, table is not temporary and partition being inserted exists, capture
+      // the list of files added. For not yet existing partitions (insert overwrite to new partition
+      // or dynamic partition inserts), the add partition event will capture the list of files added.
+      if (areEventsForDmlNeeded(tbl, oldPart)) {
+        newFiles = Collections.synchronizedList(new ArrayList<Path>());
+      }
 
       // Note: the stats for ACID tables do not have any coordination with either Hive ACID logic
       //       like txn commits, time outs, etc.; nor the lower level sync in metastore pertaining
@@ -1731,8 +1739,8 @@ public class Hive {
           Utilities.FILE_OP_LOGGER.trace("not moving " + loadPath + " to " + newPartPath + " (MM)");
         }
         assert !isAcidIUDoperation;
-        if (areEventsForDmlNeeded(tbl, oldPart)) {
-          newFiles = listFilesCreatedByQuery(loadPath, writeId, stmtId);
+        if (newFiles != null) {
+          listFilesCreatedByQuery(loadPath, writeId, stmtId, isMmTableWrite ? isInsertOverwrite : false, newFiles);
         }
         if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
           Utilities.FILE_OP_LOGGER.trace("maybe deleting stuff from " + oldPartPath
@@ -1781,8 +1789,15 @@ public class Hive {
       // or dynamic partition inserts), the add partition event will capture the list of files added.
       // Generate an insert event only if inserting into an existing partition
       // When inserting into a new partition, the add partition event takes care of insert event
-      if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() && (null != oldPart)) {
-        fireInsertEvent(tbl, partSpec, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+      if ((null != oldPart) && (null != newFiles)) {
+        if (isTxnTable) {
+          addWriteNotificationLog(tbl, partSpec, newFiles, writeId);
+        } else {
+          fireInsertEvent(tbl, partSpec, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+        }
+      } else {
+        LOG.debug("No new files were created, and is not a replace, or we're inserting into a "
+                + "partition that does not exist yet. Skipping generating INSERT event.");
       }
 
       // column stats will be inaccurate
@@ -1852,6 +1867,12 @@ public class Hive {
           }
           throw e;
         }
+
+        // For acid table, add the acid_write event with file list at the time of load itself. But
+        // it should be done after partition is created.
+        if (isTxnTable && (null != newFiles)) {
+          addWriteNotificationLog(tbl, partSpec, newFiles, writeId);
+        }
       } else {
         setStatsPropAndAlterPartition(hasFollowingStatsTask, tbl, newTPart);
       }
@@ -1906,50 +1927,47 @@ public class Hive {
   }
 
   private boolean areEventsForDmlNeeded(Table tbl, Partition oldPart) {
-    return conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() && oldPart != null;
+    // For Acid IUD, add partition is a meta data only operation. So need to add the new files added
+    // information into the TXN_WRITE_NOTIFICATION_LOG table.
+    return conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() &&
+            ((null != oldPart) || AcidUtils.isTransactionalTable(tbl));
+  }
+
+  private void listFilesInsideAcidDirectory(Path acidDir, FileSystem srcFs, List<Path> newFiles) throws IOException {
+    // list out all the files/directory in the path
+    FileStatus[] acidFiles;
+    acidFiles = srcFs.listStatus(acidDir);
+    if (acidFiles == null) {
+      LOG.debug("No files added by this query in: " + acidDir);
+      return;
+    }
+    for (FileStatus acidFile : acidFiles) {
+      // need to list out only files, ignore folders.
+      if (!acidFile.isDirectory()) {
+        newFiles.add(acidFile.getPath());
+      } else {
+        listFilesInsideAcidDirectory(acidFile.getPath(), srcFs, newFiles);
+      }
+    }
   }
 
-  private List<Path> listFilesCreatedByQuery(Path loadPath, long writeId, int stmtId) throws HiveException {
-    List<Path> newFiles = new ArrayList<Path>();
-    final String filePrefix = AcidUtils.deltaSubdir(writeId, writeId, stmtId);
-    FileStatus[] srcs;
-    FileSystem srcFs;
+  private void listFilesCreatedByQuery(Path loadPath, long writeId, int stmtId,
+                                             boolean isInsertOverwrite, List<Path> newFiles) throws HiveException {
+    Path acidDir = new Path(loadPath, AcidUtils.baseOrDeltaSubdir(isInsertOverwrite, writeId, writeId, stmtId));
     try {
-      srcFs = loadPath.getFileSystem(conf);
-      srcs = srcFs.listStatus(loadPath);
+      FileSystem srcFs = loadPath.getFileSystem(conf);
+      if (srcFs.exists(acidDir) && srcFs.isDirectory(acidDir)){
+        // list out all the files in the path
+        listFilesInsideAcidDirectory(acidDir, srcFs, newFiles);
+      } else {
+        LOG.info("directory does not exist: " + acidDir);
+        return;
+      }
     } catch (IOException e) {
       LOG.error("Error listing files", e);
       throw new HiveException(e);
     }
-    if (srcs == null) {
-      LOG.info("No sources specified: " + loadPath);
-      return newFiles;
-    }
-    PathFilter subdirFilter = null;
-
-    // Note: just like the move path, we only do one level of recursion.
-    for (FileStatus src : srcs) {
-      if (src.isDirectory()) {
-        if (subdirFilter == null) {
-          subdirFilter = new PathFilter() {
-            @Override
-            public boolean accept(Path path) {
-              return path.getName().startsWith(filePrefix);
-            }
-          };
-        }
-        try {
-          for (FileStatus srcFile : srcFs.listStatus(src.getPath(), subdirFilter)) {
-            newFiles.add(srcFile.getPath());
-          }
-        } catch (IOException e) {
-          throw new HiveException(e);
-        }
-      } else if (src.getPath().getName().startsWith(filePrefix)) {
-        newFiles.add(src.getPath());
-      }
-    }
-    return newFiles;
+    return;
   }
 
   private void setStatsPropAndAlterPartition(boolean hasFollowingStatsTask, Table tbl,
@@ -2301,13 +2319,17 @@ private void constructOneLBLocationMap(FileStatus fSta,
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_TABLE);
 
-    List<Path> newFiles = Collections.synchronizedList(new ArrayList<Path>());
+    List<Path> newFiles = null;
     Table tbl = getTable(tableName);
     assert tbl.getPath() != null : "null==getPath() for " + tbl.getTableName();
     boolean isTxnTable = AcidUtils.isTransactionalTable(tbl);
     boolean isMmTable = AcidUtils.isInsertOnlyTable(tbl);
     boolean isFullAcidTable = AcidUtils.isFullAcidTable(tbl);
 
+    if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary()) {
+      newFiles = Collections.synchronizedList(new ArrayList<Path>());
+    }
+
     // Note: this assumes both paths are qualified; which they are, currently.
     if ((isMmTable || isFullAcidTable) && loadPath.equals(tbl.getPath())) {
       /**
@@ -2319,7 +2341,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
         Utilities.FILE_OP_LOGGER.debug(
             "not moving " + loadPath + " to " + tbl.getPath() + " (MM)");
       }
-      newFiles = listFilesCreatedByQuery(loadPath, writeId, stmtId);
+
+      //new files list is required only for event notification.
+      if (newFiles != null) {
+        listFilesCreatedByQuery(loadPath, writeId, stmtId, isMmTable ? isInsertOverwrite : false, newFiles);
+      }
     } else {
       // Either a non-MM query, or a load into MM table from an external source.
       Path tblPath = tbl.getPath();
@@ -2390,10 +2416,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
     alterTable(tbl, environmentContext);
 
-    if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary()) {
-      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+    if (AcidUtils.isTransactionalTable(tbl)) {
+      addWriteNotificationLog(tbl, null, newFiles, writeId);
     } else {
-      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), null);
+      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
     }
 
     perfLogger.PerfLogEnd("MoveTask", PerfLogger.LOAD_TABLE);
@@ -2647,6 +2673,48 @@ private void constructOneLBLocationMap(FileStatus fSta,
     tpart.getSd().setLocation(partPath);
   }
 
+  private void addWriteNotificationLog(Table tbl, Map<String, String> partitionSpec,
+                                       List<Path> newFiles, Long writeId) throws HiveException {
+    if (!conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML)) {
+      LOG.debug("write notification log is ignored as dml event logging is disabled");
+      return;
+    }
+
+    if (tbl.isTemporary()) {
+      LOG.debug("write notification log is ignored as " + tbl.getTableName() + " is temporary : " + writeId);
+      return;
+    }
+
+    if (newFiles == null || newFiles.isEmpty()) {
+      LOG.debug("write notification log is ignored as file list is empty");
+      return;
+    }
+
+    LOG.debug("adding write notification log for operation " + writeId + " table " + tbl.getCompleteName() +
+                        "partition " + partitionSpec + " list of files " + newFiles);
+
+    try {
+      FileSystem fileSystem = tbl.getDataLocation().getFileSystem(conf);
+      Long txnId = SessionState.get().getTxnMgr().getCurrentTxnId();
+
+      InsertEventRequestData insertData = new InsertEventRequestData();
+      insertData.setReplace(true);
+
+      WriteNotificationLogRequest rqst = new WriteNotificationLogRequest(txnId, writeId,
+              tbl.getDbName(), tbl.getTableName(), insertData);
+      addInsertFileInformation(newFiles, fileSystem, insertData);
+
+      if (partitionSpec != null && !partitionSpec.isEmpty()) {
+        for (FieldSchema fs : tbl.getPartitionKeys()) {
+          rqst.addToPartitionVals(partitionSpec.get(fs.getName()));
+        }
+      }
+      getSynchronizedMSC().addWriteNotificationLog(rqst);
+    } catch (IOException | TException e) {
+      throw new HiveException(e);
+    }
+  }
+
   private void fireInsertEvent(Table tbl, Map<String, String> partitionSpec, boolean replace, List<Path> newFiles)
       throws HiveException {
     if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML)) {
@@ -2723,6 +2791,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       InsertEventRequestData insertData) throws IOException {
     insertData.addToFilesAdded(p.toString());
     FileChecksum cksum = fileSystem.getFileChecksum(p);
+    String acidDirPath = AcidUtils.getFirstLevelAcidDirPath(p.getParent(), fileSystem);
     // File checksum is not implemented for local filesystem (RawLocalFileSystem)
     if (cksum != null) {
       String checksumString =
@@ -2732,6 +2801,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
       // Add an empty checksum string for filesystems that don't generate one
       insertData.addToFilesAddedChecksum("");
     }
+
+    // acid dir will be present only for acid write operations.
+    if (acidDirPath != null) {
+      insertData.addToSubDirectoryList(acidDirPath);
+    }
   }
 
   public boolean dropPartition(String tblName, List<String> part_vals, boolean deleteData)
@@ -3690,7 +3764,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
                   @Override
                   public Void call() throws HiveException {
                     SessionState.setCurrentSessionState(parentSession);
-                    final String group = srcStatus.getGroup();
                     try {
                       boolean success = false;
                       if (destFs instanceof DistributedFileSystem) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
index f1c4d98..e04a0f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.metadata;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -431,11 +432,19 @@ public final class HiveUtils {
 
   public static String getReplPolicy(String dbName, String tableName) {
     if ((dbName == null) || (dbName.isEmpty())) {
-      return null;
+      return "*.*";
     } else if ((tableName == null) || (tableName.isEmpty())) {
       return dbName.toLowerCase() + ".*";
     } else {
       return dbName.toLowerCase() + "." + tableName.toLowerCase();
     }
   }
+
+  public static Path getDumpPath(Path root, String dbName, String tableName) {
+    assert (dbName != null);
+    if ((tableName != null) && (!tableName.isEmpty())) {
+      return new Path(root, dbName + "." + tableName);
+    }
+    return new Path(root, dbName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index d34de61..eb594f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.plan.ImportTableDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
+import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc.LoadFileType;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -249,9 +250,11 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new HiveException(e);
     }
 
+    boolean inReplicationScope = false;
     if ((replicationSpec != null) && replicationSpec.isInReplicationScope()){
       tblDesc.setReplicationSpec(replicationSpec);
       StatsSetupConst.setBasicStatsState(tblDesc.getTblProps(), StatsSetupConst.FALSE);
+      inReplicationScope = true;
     }
 
     if (isExternalSet) {
@@ -275,7 +278,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     for (Partition partition : partitions) {
       // TODO: this should ideally not create AddPartitionDesc per partition
       AddPartitionDesc partsDesc = getBaseAddPartitionDescFromPartition(fromPath, dbname, tblDesc, partition);
-      if ((replicationSpec != null) && replicationSpec.isInReplicationScope()){
+      if (inReplicationScope){
         StatsSetupConst.setBasicStatsState(partsDesc.getPartition(0).getPartParams(), StatsSetupConst.FALSE);
       }
       partitionDescs.add(partsDesc);
@@ -335,13 +338,14 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       //if importing into existing transactional table or will create a new transactional table
       //(because Export was done from transactional table), need a writeId
       // Explain plan doesn't open a txn and hence no need to allocate write id.
-      if (x.getCtx().getExplainConfig() == null) {
+      // In replication flow, no need to allocate write id. It will be allocated using the alloc write id event.
+      if (x.getCtx().getExplainConfig() == null && !inReplicationScope) {
         writeId = txnMgr.getTableWriteId(tblDesc.getDatabaseName(), tblDesc.getTableName());
         stmtId = txnMgr.getStmtIdAndIncrement();
       }
     }
 
-    if (!replicationSpec.isInReplicationScope()) {
+    if (!inReplicationScope) {
       createRegularImportTasks(
           tblDesc, partitionDescs,
           isPartSpecSet, replicationSpec, table,
@@ -390,7 +394,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     Path dataPath = new Path(fromURI.toString(), EximUtil.DATA_PATH_NAME);
     Path destPath = null, loadPath = null;
     LoadFileType lft;
-    if (AcidUtils.isTransactionalTable(table)) {
+    if (AcidUtils.isTransactionalTable(table) && !replicationSpec.isInReplicationScope()) {
       String mmSubdir = replace ? AcidUtils.baseDir(writeId)
           : AcidUtils.deltaSubdir(writeId, writeId, stmtId);
       destPath = new Path(tgtPath, mmSubdir);
@@ -428,13 +432,26 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       copyTask = TaskFactory.get(new CopyWork(dataPath, destPath, false));
     }
 
-    LoadTableDesc loadTableWork = new LoadTableDesc(
-        loadPath, Utilities.getTableDesc(table), new TreeMap<>(), lft, writeId);
-    loadTableWork.setStmtId(stmtId);
+    MoveWork moveWork = new MoveWork(x.getInputs(), x.getOutputs(), null, null, false);
+
+
+    if (replicationSpec.isInReplicationScope() && AcidUtils.isTransactionalTable(table)) {
+      LoadMultiFilesDesc loadFilesWork = new LoadMultiFilesDesc(
+              Collections.singletonList(destPath),
+              Collections.singletonList(tgtPath),
+              true, null, null);
+      moveWork.setMultiFilesDesc(loadFilesWork);
+      moveWork.setNeedCleanTarget(false);
+    } else {
+      LoadTableDesc loadTableWork = new LoadTableDesc(
+              loadPath, Utilities.getTableDesc(table), new TreeMap<>(), lft, writeId);
+      loadTableWork.setStmtId(stmtId);
+      moveWork.setLoadTableWork(loadTableWork);
+    }
+
     //if Importing into existing table, FileFormat is checked by
     // ImportSemanticAnalzyer.checked checkTable()
-    MoveWork mv = new MoveWork(x.getInputs(), x.getOutputs(), loadTableWork, null, false);
-    Task<?> loadTableTask = TaskFactory.get(mv, x.getConf());
+    Task<?> loadTableTask = TaskFactory.get(moveWork, x.getConf());
     copyTask.addDependentTask(loadTableTask);
     x.getTasks().add(copyTask);
     return loadTableTask;
@@ -498,8 +515,10 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           + partSpecToString(partSpec.getPartSpec())
           + " with source location: " + srcLocation);
       Path tgtLocation = new Path(partSpec.getLocation());
-      Path destPath = !AcidUtils.isTransactionalTable(table.getParameters()) ?
-          x.getCtx().getExternalTmpPath(tgtLocation)
+      //Replication scope the write id will be invalid
+      Boolean useStagingDirectory = !AcidUtils.isTransactionalTable(table.getParameters()) ||
+              replicationSpec.isInReplicationScope();
+      Path destPath =  useStagingDirectory ? x.getCtx().getExternalTmpPath(tgtLocation)
           : new Path(tgtLocation, AcidUtils.deltaSubdir(writeId, writeId, stmtId));
       Path moveTaskSrc =  !AcidUtils.isTransactionalTable(table.getParameters()) ? destPath : tgtLocation;
       if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
@@ -523,17 +542,29 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       Task<?> addPartTask = TaskFactory.get(
               new DDLWork(x.getInputs(), x.getOutputs(), addPartitionDesc), x.getConf());
 
+      MoveWork moveWork = new MoveWork(x.getInputs(), x.getOutputs(),
+              null, null, false);
+
       // Note: this sets LoadFileType incorrectly for ACID; is that relevant for import?
       //       See setLoadFileType and setIsAcidIow calls elsewhere for an example.
-      LoadTableDesc loadTableWork = new LoadTableDesc(moveTaskSrc, Utilities.getTableDesc(table),
-          partSpec.getPartSpec(),
-          replicationSpec.isReplace() ? LoadFileType.REPLACE_ALL : LoadFileType.OVERWRITE_EXISTING,
-              writeId);
-      loadTableWork.setStmtId(stmtId);
-      loadTableWork.setInheritTableSpecs(false);
-      Task<?> loadPartTask = TaskFactory.get(
-              new MoveWork(x.getInputs(), x.getOutputs(), loadTableWork, null, false),
-              x.getConf());
+      if (replicationSpec.isInReplicationScope() && AcidUtils.isTransactionalTable(tblDesc.getTblProps())) {
+        LoadMultiFilesDesc loadFilesWork = new LoadMultiFilesDesc(
+                Collections.singletonList(destPath),
+                Collections.singletonList(tgtLocation),
+                true, null, null);
+        moveWork.setMultiFilesDesc(loadFilesWork);
+        moveWork.setNeedCleanTarget(false);
+      } else {
+        LoadTableDesc loadTableWork = new LoadTableDesc(moveTaskSrc, Utilities.getTableDesc(table),
+                partSpec.getPartSpec(),
+                replicationSpec.isReplace() ? LoadFileType.REPLACE_ALL : LoadFileType.OVERWRITE_EXISTING,
+                writeId);
+        loadTableWork.setStmtId(stmtId);
+        loadTableWork.setInheritTableSpecs(false);
+        moveWork.setLoadTableWork(loadTableWork);
+      }
+
+      Task<?> loadPartTask = TaskFactory.get(moveWork, x.getConf());
       copyTask.addDependentTask(loadPartTask);
       addPartTask.addDependentTask(loadPartTask);
       x.getTasks().add(copyTask);
@@ -1005,7 +1036,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
             t.addDependentTask(
                 addSinglePartition(fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, x, writeId, stmtId));
             if (updatedMetadata != null) {
-              updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+              updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                      addPartitionDesc.getPartition(0).getPartSpec());
             }
           }
         } else {
@@ -1057,13 +1089,15 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
               x.getTasks().add(addSinglePartition(
                   fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, x, writeId, stmtId));
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
             } else {
               x.getTasks().add(alterSinglePartition(
                       fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, null, x));
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
             }
           } else {
@@ -1078,7 +1112,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
                     fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, ptn, x));
               }
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
               if (lockType == WriteEntity.WriteType.DDL_NO_LOCK){
                 lockType = WriteEntity.WriteType.DDL_SHARED;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 576f337..1271799 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -7309,7 +7309,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
         try {
           if (ctx.getExplainConfig() != null) {
-            writeId = 0L; // For explain plan, txn won't be opened and doesn't make sense to allocate write id
+            writeId = null; // For explain plan, txn won't be opened and doesn't make sense to allocate write id
           } else {
             if (isMmTable) {
               writeId = txnMgr.getTableWriteId(dest_tab.getDbName(), dest_tab.getTableName());
@@ -7324,6 +7324,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         boolean isReplace = !qb.getParseInfo().isInsertIntoTable(
             dest_tab.getDbName(), dest_tab.getTableName());
         ltd = new LoadTableDesc(queryTmpdir, table_desc, dpCtx, acidOp, isReplace, writeId);
+        if (writeId != null) {
+          ltd.setStmtId(txnMgr.getCurrentStmtId());
+        }
         // For Acid table, Insert Overwrite shouldn't replace the table content. We keep the old
         // deltas and base and leave them up to the cleaner to clean up
         boolean isInsertInto = qb.getParseInfo().isInsertIntoTable(
@@ -7419,6 +7422,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         throw new SemanticException("Failed to allocate write Id", ex);
       }
       ltd = new LoadTableDesc(queryTmpdir, table_desc, dest_part.getSpec(), acidOp, writeId);
+      if (writeId != null) {
+        ltd.setStmtId(txnMgr.getCurrentStmtId());
+      }
       // For the current context for generating File Sink Operator, it is either INSERT INTO or INSERT OVERWRITE.
       // So the next line works.
       boolean isInsertInto = !qb.getParseInfo().isDestToOpTypeInsertOverwrite(dest);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index ce7e65a..8df2904 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -179,9 +179,23 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     String newTableName = getTmptTableNameForExport(exportTable); //this is db.table
     Map<String, String> tblProps = new HashMap<>();
     tblProps.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, Boolean.FALSE.toString());
+    String location;
+
+    // for temporary tables we set the location to something in the session's scratch dir
+    // it has the same life cycle as the tmp table
+    try {
+      // Generate a unique ID for temp table path.
+      // This path will be fixed for the life of the temp table.
+      Path path = new Path(SessionState.getTempTableSpace(conf), UUID.randomUUID().toString());
+      path = Warehouse.getDnsPath(path, conf);
+      location = path.toString();
+    } catch (MetaException err) {
+      throw new SemanticException("Error while generating temp table path:", err);
+    }
+
     CreateTableLikeDesc ctlt = new CreateTableLikeDesc(newTableName,
         false, true, null,
-        null, null, null, null,
+        null, location, null, null,
         tblProps,
         true, //important so we get an exception on name collision
         Warehouse.getQualifiedName(exportTable.getTTable()), false);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
index 61bf6b9..75dcaa3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
@@ -400,7 +400,7 @@ public class CopyUtils {
     return result;
   }
 
-  private Path getCopyDestination(ReplChangeManager.FileInfo fileInfo, Path destRoot) {
+  public static Path getCopyDestination(ReplChangeManager.FileInfo fileInfo, Path destRoot) {
     if (fileInfo.getSubDir() == null) {
       return destRoot;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
index c0701c5..62d699f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
@@ -186,10 +186,6 @@ public class Utils {
         return false;
       }
 
-      boolean isAcidTable = AcidUtils.isTransactionalTable(tableHandle);
-      if (isAcidTable) {
-        return hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_INCLUDE_ACID_TABLES);
-      }
       return !tableHandle.isTemporary();
     }
     return true;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
index db97d7c..f04cd93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
@@ -18,9 +18,27 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.parse.EximUtil;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
 import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
+import org.apache.hadoop.fs.FileSystem;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
 
 class CommitTxnHandler extends AbstractEventHandler {
 
@@ -28,11 +46,116 @@ class CommitTxnHandler extends AbstractEventHandler {
     super(event);
   }
 
+  private BufferedWriter writer(Context withinContext, Path dataPath) throws IOException {
+    Path filesPath = new Path(dataPath, EximUtil.FILES_NAME);
+    FileSystem fs = dataPath.getFileSystem(withinContext.hiveConf);
+    return new BufferedWriter(new OutputStreamWriter(fs.create(filesPath)));
+  }
+
+  private void writeDumpFiles(Context withinContext, Iterable<String> files, Path dataPath) throws IOException {
+    // encoded filename/checksum of files, write into _files
+    try (BufferedWriter fileListWriter = writer(withinContext, dataPath)) {
+      for (String file : files) {
+        fileListWriter.write(file + "\n");
+      }
+    }
+  }
+
+  private void createDumpFile(Context withinContext, org.apache.hadoop.hive.ql.metadata.Table qlMdTable,
+                  List<Partition> qlPtns, List<List<String>> fileListArray) throws IOException, SemanticException {
+    if (fileListArray == null || fileListArray.isEmpty()) {
+      return;
+    }
+
+    Path metaDataPath = new Path(withinContext.eventRoot, EximUtil.METADATA_NAME);
+    withinContext.replicationSpec.setIsReplace(true);
+    EximUtil.createExportDump(metaDataPath.getFileSystem(withinContext.hiveConf), metaDataPath,
+            qlMdTable, qlPtns,
+            withinContext.replicationSpec,
+            withinContext.hiveConf);
+
+    if ((null == qlPtns) || qlPtns.isEmpty()) {
+      Path dataPath = new Path(withinContext.eventRoot, EximUtil.DATA_PATH_NAME);
+      writeDumpFiles(withinContext, fileListArray.get(0), dataPath);
+    } else {
+      for (int idx = 0; idx < qlPtns.size(); idx++) {
+        Path dataPath = new Path(withinContext.eventRoot, qlPtns.get(idx).getName());
+        writeDumpFiles(withinContext, fileListArray.get(idx), dataPath);
+      }
+    }
+  }
+
+  private void createDumpFileForTable(Context withinContext, org.apache.hadoop.hive.ql.metadata.Table qlMdTable,
+                    List<Partition> qlPtns, List<List<String>> fileListArray) throws IOException, SemanticException {
+    Path newPath = HiveUtils.getDumpPath(withinContext.eventRoot, qlMdTable.getDbName(), qlMdTable.getTableName());
+    Context context = new Context(withinContext);
+    context.setEventRoot(newPath);
+    createDumpFile(context, qlMdTable, qlPtns, fileListArray);
+  }
+
   @Override
   public void handle(Context withinContext) throws Exception {
     LOG.info("Processing#{} COMMIT_TXN message : {}", fromEventId(), event.getMessage());
+    String payload = event.getMessage();
+
+    if (!withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      CommitTxnMessage commitTxnMessage = deserializer.getCommitTxnMessage(event.getMessage());
+
+      String contextDbName =  withinContext.dbName == null ? null :
+              StringUtils.normalizeIdentifier(withinContext.dbName);
+      String contextTableName =  withinContext.tableName == null ? null :
+              StringUtils.normalizeIdentifier(withinContext.tableName);
+      List<WriteEventInfo> writeEventInfoList = HiveMetaStore.HMSHandler.getMSForConf(withinContext.hiveConf).
+              getAllWriteEventInfo(commitTxnMessage.getTxnId(), contextDbName, contextTableName);
+      int numEntry = (writeEventInfoList != null ? writeEventInfoList.size() : 0);
+      if (numEntry != 0) {
+        commitTxnMessage.addWriteEventInfo(writeEventInfoList);
+        payload = commitTxnMessage.toString();
+        LOG.debug("payload for commit txn event : " + payload);
+      }
+
+      org.apache.hadoop.hive.ql.metadata.Table qlMdTablePrev = null;
+      org.apache.hadoop.hive.ql.metadata.Table qlMdTable = null;
+      List<Partition> qlPtns = new ArrayList<>();
+      List<List<String>> filesTobeAdded = new ArrayList<>();
+
+      // The below loop creates dump directory for each table. It reads through the list of write notification events,
+      // groups the entries per table and creates the lists of files to be replicated. The event directory in the dump
+      // path will have subdirectory for each table. This folder will have metadata for the table and the list of files
+      // to be replicated. The entries are added in the table with txn id, db name,table name, partition name
+      // combination as primary key, so the entries with same table will come together. Only basic table metadata is
+      // used during import, so we need not dump the latest table metadata.
+      for (int idx = 0; idx < numEntry; idx++) {
+        qlMdTable = new org.apache.hadoop.hive.ql.metadata.Table(commitTxnMessage.getTableObj(idx));
+        if (qlMdTablePrev == null) {
+          qlMdTablePrev = qlMdTable;
+        }
+
+        // one dump directory per table
+        if (!qlMdTablePrev.getCompleteName().equals(qlMdTable.getCompleteName())) {
+          createDumpFileForTable(withinContext, qlMdTablePrev, qlPtns, filesTobeAdded);
+          qlPtns = new ArrayList<>();
+          filesTobeAdded = new ArrayList<>();
+          qlMdTablePrev = qlMdTable;
+        }
+
+        if (qlMdTable.isPartitioned() && (null != commitTxnMessage.getPartitionObj(idx))) {
+          qlPtns.add(new org.apache.hadoop.hive.ql.metadata.Partition(qlMdTable,
+                  commitTxnMessage.getPartitionObj(idx)));
+        }
+
+        filesTobeAdded.add(Lists.newArrayList(
+                ReplChangeManager.getListFromSeparatedString(commitTxnMessage.getFiles(idx))));
+      }
+
+      //Dump last table in the list
+      if (qlMdTablePrev != null) {
+        createDumpFileForTable(withinContext, qlMdTablePrev, qlPtns, filesTobeAdded);
+      }
+    }
+
     DumpMetaData dmd = withinContext.createDmd(this);
-    dmd.setPayload(event.getMessage());
+    dmd.setPayload(payload);
     dmd.write();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
index c0fa7b2..ec35f4e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
@@ -35,18 +35,37 @@ public interface EventHandler {
   DumpType dumpType();
 
   class Context {
-    final Path eventRoot, cmRoot;
+    Path eventRoot;
+    final Path  cmRoot;
     final Hive db;
     final HiveConf hiveConf;
     final ReplicationSpec replicationSpec;
+    final String dbName;
+    final String tableName;
 
     public Context(Path eventRoot, Path cmRoot, Hive db, HiveConf hiveConf,
-        ReplicationSpec replicationSpec) {
+        ReplicationSpec replicationSpec, String dbName, String tableName) {
       this.eventRoot = eventRoot;
       this.cmRoot = cmRoot;
       this.db = db;
       this.hiveConf = hiveConf;
       this.replicationSpec = replicationSpec;
+      this.dbName = dbName;
+      this.tableName = tableName;
+    }
+
+    public Context(Context other) {
+      this.eventRoot = other.eventRoot;
+      this.cmRoot = other.cmRoot;
+      this.db = other.db;
+      this.hiveConf = other.hiveConf;
+      this.replicationSpec = other.replicationSpec;
+      this.dbName = other.dbName;
+      this.tableName = other.tableName;
+    }
+
+    public void setEventRoot(Path eventRoot) {
+      this.eventRoot = eventRoot;
     }
 
     DumpMetaData createDmd(EventHandler eventHandler) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
index 5ac3af0..cf3822a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.parse.EximUtil;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
@@ -53,6 +54,9 @@ class InsertHandler extends AbstractEventHandler {
       return;
     }
 
+    // In case of ACID tables, insert event should not have fired.
+    assert(!AcidUtils.isTransactionalTable(qlMdTable));
+
     List<Partition> qlPtns = null;
     if (qlMdTable.isPartitioned() && (null != insertMsg.getPtnObj())) {
       qlPtns = Collections.singletonList(partitionObject(qlMdTable, insertMsg));

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
index d76f419..614e071 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.load;
 
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hive.common.util.HiveStringUtils;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.List;
 
@@ -25,52 +28,113 @@ import java.util.List;
  * Utility class to help track and return the metadata which are updated by repl load
  */
 public class UpdatedMetaDataTracker {
-  private String replState;
-  private String dbName;
-  private String tableName;
-  private List<Map <String, String>> partitionsList;
 
-  public UpdatedMetaDataTracker() {
-    this.replState = null;
-    this.dbName = null;
-    this.tableName = null;
-    this.partitionsList = new ArrayList<>();
+  /**
+   * Utility class to store replication state of a table.
+   */
+  public static class UpdateMetaData {
+    private String replState;
+    private String dbName;
+    private String tableName;
+    private List<Map <String, String>> partitionsList;
+
+    UpdateMetaData(String replState, String dbName, String tableName, Map <String, String> partSpec) {
+      this.replState = replState;
+      this.dbName = dbName;
+      this.tableName = tableName;
+      this.partitionsList = new ArrayList<>();
+      if (partSpec != null) {
+        this.partitionsList.add(partSpec);
+      }
+    }
+
+    public String getReplState() {
+      return replState;
+    }
+
+    public String getDbName() {
+      return dbName;
+    }
+
+    public String getTableName() {
+      return tableName;
+    }
+
+    public List<Map <String, String>> getPartitionsList() {
+      return partitionsList;
+    }
+
+    public void addPartition(Map<String, String> partSpec) {
+      this.partitionsList.add(partSpec);
+    }
   }
 
-  public void copyUpdatedMetadata(UpdatedMetaDataTracker other) {
-    this.replState = other.replState;
-    this.dbName = other.dbName;
-    this.tableName = other.tableName;
-    this.partitionsList = other.getPartitions();
+  private List<UpdateMetaData> updateMetaDataList;
+  private Map<String, Integer> updateMetaDataMap;
+
+  public UpdatedMetaDataTracker() {
+    updateMetaDataList = new ArrayList<>();
+    updateMetaDataMap = new HashMap<>();
   }
 
-  public void set(String replState, String dbName, String tableName, Map <String, String> partSpec) {
-    this.replState = replState;
-    this.dbName = dbName;
-    this.tableName = tableName;
-    if (partSpec != null) {
-      addPartition(partSpec);
+  public void copyUpdatedMetadata(UpdatedMetaDataTracker other) {
+    int size = updateMetaDataList.size();
+    for (UpdateMetaData updateMetaDataOther : other.updateMetaDataList) {
+      String key = getKey(normalizeIdentifier(updateMetaDataOther.getDbName()),
+              normalizeIdentifier(updateMetaDataOther.getTableName()));
+      Integer idx = updateMetaDataMap.get(key);
+      if (idx == null) {
+        updateMetaDataList.add(updateMetaDataOther);
+        updateMetaDataMap.put(key, size++);
+      } else if (updateMetaDataOther.partitionsList != null && updateMetaDataOther.partitionsList.size() != 0) {
+        UpdateMetaData updateMetaData = updateMetaDataList.get(idx);
+        for (Map<String, String> partSpec : updateMetaDataOther.partitionsList) {
+          updateMetaData.addPartition(partSpec);
+        }
+      }
     }
   }
 
-  public void addPartition(Map <String, String> partSpec) {
-    partitionsList.add(partSpec);
+  public void set(String replState, String dbName, String tableName, Map <String, String> partSpec)
+          throws SemanticException {
+    if (dbName == null) {
+      throw new SemanticException("db name can not be null");
+    }
+    String key = getKey(normalizeIdentifier(dbName), normalizeIdentifier(tableName));
+    Integer idx = updateMetaDataMap.get(key);
+    if (idx == null) {
+      updateMetaDataList.add(new UpdateMetaData(replState, dbName, tableName, partSpec));
+      updateMetaDataMap.put(key, updateMetaDataList.size() - 1);
+    } else {
+      updateMetaDataList.get(idx).addPartition(partSpec);
+    }
   }
 
-  public String getReplicationState() {
-    return replState;
+  public void addPartition(String dbName, String tableName, Map <String, String> partSpec) throws SemanticException {
+    if (dbName == null) {
+      throw new SemanticException("db name can not be null");
+    }
+    String key = getKey(normalizeIdentifier(dbName), normalizeIdentifier(tableName));
+    Integer idx = updateMetaDataMap.get(key);
+    if (idx == null) {
+      throw new SemanticException("add partition to metadata map failed as list is not yet set for table : " + key);
+    }
+    updateMetaDataList.get(idx).addPartition(partSpec);
   }
 
-  public String getDatabase() {
-    return dbName;
+  public List<UpdateMetaData> getUpdateMetaDataList() {
+    return updateMetaDataList;
   }
 
-  public String getTable() {
-    return tableName;
+  private String getKey(String dbName, String tableName) {
+    if (tableName == null) {
+      return dbName + ".*";
+    }
+    return dbName + "." + tableName;
   }
 
-  public List<Map <String, String>> getPartitions() {
-    return partitionsList;
+  private String normalizeIdentifier(String name) {
+    return name == null ? null : HiveStringUtils.normalizeIdentifier(name);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
index afc7426..d3f3306 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
@@ -48,7 +48,12 @@ public class AbortTxnHandler extends AbstractMessageHandler {
                 msg.getTxnId(), ReplTxnWork.OperationType.REPL_ABORT_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Abort txn task : {}", abortTxnTask.getId());
     return Collections.singletonList(abortTxnTask);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
index 9bdbf64..63f2577 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
@@ -52,7 +52,7 @@ public class AllocWriteIdHandler extends AbstractMessageHandler {
             .getTableName());
 
     // Repl policy should be created based on the table name in context.
-    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(dbName, context.tableName), dbName, tableName,
+    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), dbName, tableName,
         ReplTxnWork.OperationType.REPL_ALLOC_WRITE_ID, msg.getTxnToWriteIdList(), context.eventOnlyReplicationSpec());
 
     Task<? extends Serializable> allocWriteIdTask = TaskFactory.get(work, context.hiveConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
index d25102e..0619bd3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.load.message;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
+import org.apache.hadoop.hive.ql.exec.repl.util.AddDependencyToLeaves;
+import org.apache.hadoop.hive.ql.exec.util.DAGTraversal;
+import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.ReplTxnWork;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -25,7 +30,7 @@ import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import java.io.Serializable;
-import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -35,20 +40,75 @@ import java.util.List;
 public class CommitTxnHandler extends AbstractMessageHandler {
   @Override
   public List<Task<? extends Serializable>> handle(Context context)
-      throws SemanticException {
+          throws SemanticException {
     if (!AcidUtils.isAcidEnabled(context.hiveConf)) {
       context.log.error("Cannot load transaction events as acid is not enabled");
       throw new SemanticException("Cannot load transaction events as acid is not enabled");
     }
 
     CommitTxnMessage msg = deserializer.getCommitTxnMessage(context.dmd.getPayload());
-    Task<ReplTxnWork> commitTxnTask = TaskFactory.get(
-        new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName, context.tableName,
-              msg.getTxnId(), ReplTxnWork.OperationType.REPL_COMMIT_TXN, context.eventOnlyReplicationSpec()),
-        context.hiveConf
-    );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    int numEntry = (msg.getTables() == null ? 0 : msg.getTables().size());
+    List<Task<? extends Serializable>> tasks = new ArrayList<>();
+    String dbName = context.dbName;
+    String tableNamePrev = null;
+    String tblName = context.tableName;
+
+    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName,
+      context.tableName, msg.getTxnId(), ReplTxnWork.OperationType.REPL_COMMIT_TXN, context.eventOnlyReplicationSpec());
+
+    if (numEntry > 0) {
+      context.log.debug("Commit txn handler for txnid " + msg.getTxnId() + " databases : " + msg.getDatabases() +
+              " tables : " + msg.getTables() + " partitions : " + msg.getPartitions() + " files : " +
+              msg.getFilesList() + " write ids : " + msg.getWriteIds());
+    }
+
+    for (int idx = 0; idx < numEntry; idx++) {
+      String actualTblName = msg.getTables().get(idx);
+      String actualDBName = msg.getDatabases().get(idx);
+      String completeName = Table.getCompleteName(actualDBName, actualTblName);
+
+      // One import task per table. Events for same table are kept together in one dump directory during dump and are
+      // grouped together in commit txn message.
+      if (tableNamePrev == null || !(completeName.equals(tableNamePrev))) {
+        // The data location is created by source, so the location should be formed based on the table name in msg.
+        Path location = HiveUtils.getDumpPath(new Path(context.location), actualDBName, actualTblName);
+        tblName = context.isTableNameEmpty() ? actualTblName : context.tableName;
+        // for warehouse level dump, use db name from write event
+        dbName = (context.isDbNameEmpty() ? actualDBName : context.dbName);
+        Context currentContext = new Context(context, dbName, tblName);
+        currentContext.setLocation(location.toUri().toString());
+
+        // Piggybacking in Import logic for now
+        TableHandler tableHandler = new TableHandler();
+        tasks.addAll((tableHandler.handle(currentContext)));
+        readEntitySet.addAll(tableHandler.readEntities());
+        writeEntitySet.addAll(tableHandler.writeEntities());
+        getUpdatedMetadata().copyUpdatedMetadata(tableHandler.getUpdatedMetadata());
+        tableNamePrev = completeName;
+      }
+
+      try {
+        WriteEventInfo writeEventInfo = new WriteEventInfo(msg.getWriteIds().get(idx),
+                dbName, tblName, msg.getFiles(idx));
+        if (msg.getPartitions().get(idx) != null && !msg.getPartitions().get(idx).isEmpty()) {
+          writeEventInfo.setPartition(msg.getPartitions().get(idx));
+        }
+        work.addWriteEventInfo(writeEventInfo);
+      } catch (Exception e) {
+        throw new SemanticException("Failed to extract write event info from commit txn message : " + e.getMessage());
+      }
+    }
+
+    Task<ReplTxnWork> commitTxnTask = TaskFactory.get(work, context.hiveConf);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Commit txn task : {}", commitTxnTask.getId());
-    return Collections.singletonList(commitTxnTask);
+    DAGTraversal.traverse(tasks, new AddDependencyToLeaves(commitTxnTask));
+    return tasks;
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
index ef4a901..cdf51dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
@@ -46,8 +46,8 @@ public interface MessageHandler {
   UpdatedMetaDataTracker getUpdatedMetadata();
 
   class Context {
-    public String dbName;
-    public final String tableName, location;
+    public String location;
+    public final String tableName, dbName;
     public final Task<? extends Serializable> precursor;
     public DumpMetaData dmd;
     final HiveConf hiveConf;
@@ -101,5 +101,9 @@ public interface MessageHandler {
     public HiveTxnManager getTxnMgr() {
       return nestedContext.getHiveTxnManager();
     }
+
+    public void setLocation(String location) {
+      this.location = location;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
index 190e021..5dcc44e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
@@ -47,7 +47,12 @@ public class OpenTxnHandler extends AbstractMessageHandler {
                 msg.getTxnIds(), ReplTxnWork.OperationType.REPL_OPEN_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Open txn task : {}", openTxnTask.getId());
     return Collections.singletonList(openTxnTask);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
index 9a1e3a1..47a56d5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
@@ -40,6 +40,7 @@ public class MoveWork implements Serializable {
   private LoadMultiFilesDesc loadMultiFilesWork;
   private boolean checkFileFormat;
   private boolean srcLocal;
+  private boolean needCleanTarget;
 
   /**
    * ReadEntitites that are passed to the hooks.
@@ -63,6 +64,7 @@ public class MoveWork implements Serializable {
   private MoveWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) {
     this.inputs = inputs;
     this.outputs = outputs;
+    this.needCleanTarget = true;
   }
 
   public MoveWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -93,6 +95,7 @@ public class MoveWork implements Serializable {
     srcLocal = o.isSrcLocal();
     inputs = o.getInputs();
     outputs = o.getOutputs();
+    needCleanTarget = o.needCleanTarget;
   }
 
   @Explain(displayName = "tables", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
@@ -153,5 +156,12 @@ public class MoveWork implements Serializable {
   public void setSrcLocal(boolean srcLocal) {
     this.srcLocal = srcLocal;
   }
-  
+
+  public boolean isNeedCleanTarget() {
+    return needCleanTarget;
+  }
+
+  public void setNeedCleanTarget(boolean needCleanTarget) {
+    this.needCleanTarget = needCleanTarget;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
index 3c853c9..a6ab836 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.Serializable;
 
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
@@ -40,6 +42,7 @@ public class ReplTxnWork implements Serializable {
   private List<Long> txnIds;
   private List<TxnToWriteId> txnToWriteIdList;
   private ReplicationSpec replicationSpec;
+  private List<WriteEventInfo> writeEventInfos;
 
   /**
    * OperationType.
@@ -60,6 +63,7 @@ public class ReplTxnWork implements Serializable {
     this.replPolicy = replPolicy;
     this.txnToWriteIdList = txnToWriteIdList;
     this.replicationSpec = replicationSpec;
+    this.writeEventInfos = null;
   }
 
   public ReplTxnWork(String replPolicy, String dbName, String tableName, List<Long> txnIds, OperationType type,
@@ -86,6 +90,13 @@ public class ReplTxnWork implements Serializable {
     this.operation = type;
   }
 
+  public void addWriteEventInfo(WriteEventInfo writeEventInfo) {
+    if (this.writeEventInfos == null) {
+      this.writeEventInfos = new ArrayList<>();
+    }
+    this.writeEventInfos.add(writeEventInfo);
+  }
+
   public List<Long> getTxnIds() {
     return txnIds;
   }
@@ -121,4 +132,8 @@ public class ReplTxnWork implements Serializable {
   public ReplicationSpec getReplicationSpec() {
     return replicationSpec;
   }
+
+  public List<WriteEventInfo> getWriteEventInfos() {
+    return writeEventInfos;
+  }
 }


[13/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 1d57aee..352f5c7 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -187,6 +187,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) = 0;
   virtual void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) = 0;
   virtual void flushCache() = 0;
+  virtual void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) = 0;
   virtual void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) = 0;
   virtual void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) = 0;
   virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0;
@@ -780,6 +781,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void flushCache() {
     return;
   }
+  void add_write_notification_log(WriteNotificationLogResponse& /* _return */, const WriteNotificationLogRequest& /* rqst */) {
+    return;
+  }
   void cm_recycle(CmRecycleResponse& /* _return */, const CmRecycleRequest& /* request */) {
     return;
   }
@@ -21228,6 +21232,110 @@ class ThriftHiveMetastore_flushCache_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_add_write_notification_log_args__isset {
+  _ThriftHiveMetastore_add_write_notification_log_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_add_write_notification_log_args__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_args {
+ public:
+
+  ThriftHiveMetastore_add_write_notification_log_args(const ThriftHiveMetastore_add_write_notification_log_args&);
+  ThriftHiveMetastore_add_write_notification_log_args& operator=(const ThriftHiveMetastore_add_write_notification_log_args&);
+  ThriftHiveMetastore_add_write_notification_log_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_args() throw();
+  WriteNotificationLogRequest rqst;
+
+  _ThriftHiveMetastore_add_write_notification_log_args__isset __isset;
+
+  void __set_rqst(const WriteNotificationLogRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_notification_log_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_notification_log_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_notification_log_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_write_notification_log_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_pargs() throw();
+  const WriteNotificationLogRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_notification_log_result__isset {
+  _ThriftHiveMetastore_add_write_notification_log_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_add_write_notification_log_result__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_result {
+ public:
+
+  ThriftHiveMetastore_add_write_notification_log_result(const ThriftHiveMetastore_add_write_notification_log_result&);
+  ThriftHiveMetastore_add_write_notification_log_result& operator=(const ThriftHiveMetastore_add_write_notification_log_result&);
+  ThriftHiveMetastore_add_write_notification_log_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_result() throw();
+  WriteNotificationLogResponse success;
+
+  _ThriftHiveMetastore_add_write_notification_log_result__isset __isset;
+
+  void __set_success(const WriteNotificationLogResponse& val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_notification_log_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_notification_log_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_notification_log_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_notification_log_presult__isset {
+  _ThriftHiveMetastore_add_write_notification_log_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_add_write_notification_log_presult__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_presult() throw();
+  WriteNotificationLogResponse* success;
+
+  _ThriftHiveMetastore_add_write_notification_log_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_cm_recycle_args__isset {
   _ThriftHiveMetastore_cm_recycle_args__isset() : request(false) {}
   bool request :1;
@@ -26718,6 +26826,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void flushCache();
   void send_flushCache();
   void recv_flushCache();
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst);
+  void send_add_write_notification_log(const WriteNotificationLogRequest& rqst);
+  void recv_add_write_notification_log(WriteNotificationLogResponse& _return);
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request);
   void send_cm_recycle(const CmRecycleRequest& request);
   void recv_cm_recycle(CmRecycleResponse& _return);
@@ -27019,6 +27130,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_add_write_notification_log(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -27230,6 +27342,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_notification_events_count"] = &ThriftHiveMetastoreProcessor::process_get_notification_events_count;
     processMap_["fire_listener_event"] = &ThriftHiveMetastoreProcessor::process_fire_listener_event;
     processMap_["flushCache"] = &ThriftHiveMetastoreProcessor::process_flushCache;
+    processMap_["add_write_notification_log"] = &ThriftHiveMetastoreProcessor::process_add_write_notification_log;
     processMap_["cm_recycle"] = &ThriftHiveMetastoreProcessor::process_cm_recycle;
     processMap_["get_file_metadata_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr;
     processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata;
@@ -28884,6 +28997,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->flushCache();
   }
 
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_write_notification_log(_return, rqst);
+    }
+    ifaces_[i]->add_write_notification_log(_return, rqst);
+    return;
+  }
+
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -29805,6 +29928,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void flushCache();
   int32_t send_flushCache();
   void recv_flushCache(const int32_t seqid);
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst);
+  int32_t send_add_write_notification_log(const WriteNotificationLogRequest& rqst);
+  void recv_add_write_notification_log(WriteNotificationLogResponse& _return, const int32_t seqid);
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request);
   int32_t send_cm_recycle(const CmRecycleRequest& request);
   void recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index d45ec81..789c150 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -847,6 +847,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("flushCache\n");
   }
 
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) {
+    // Your implementation goes here
+    printf("add_write_notification_log\n");
+  }
+
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) {
     // Your implementation goes here
     printf("cm_recycle\n");


[05/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index a29ebb7..93b5780 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -16558,6 +16558,10 @@ class CommitTxnRequest {
    * @var string
    */
   public $replPolicy = null;
+  /**
+   * @var \metastore\WriteEventInfo[]
+   */
+  public $writeEventInfos = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -16570,6 +16574,15 @@ class CommitTxnRequest {
           'var' => 'replPolicy',
           'type' => TType::STRING,
           ),
+        3 => array(
+          'var' => 'writeEventInfos',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\WriteEventInfo',
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -16579,6 +16592,9 @@ class CommitTxnRequest {
       if (isset($vals['replPolicy'])) {
         $this->replPolicy = $vals['replPolicy'];
       }
+      if (isset($vals['writeEventInfos'])) {
+        $this->writeEventInfos = $vals['writeEventInfos'];
+      }
     }
   }
 
@@ -16615,6 +16631,24 @@ class CommitTxnRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->writeEventInfos = array();
+            $_size523 = 0;
+            $_etype526 = 0;
+            $xfer += $input->readListBegin($_etype526, $_size523);
+            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            {
+              $elem528 = null;
+              $elem528 = new \metastore\WriteEventInfo();
+              $xfer += $elem528->read($input);
+              $this->writeEventInfos []= $elem528;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -16638,6 +16672,236 @@ class CommitTxnRequest {
       $xfer += $output->writeString($this->replPolicy);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->writeEventInfos !== null) {
+      if (!is_array($this->writeEventInfos)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('writeEventInfos', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->writeEventInfos));
+        {
+          foreach ($this->writeEventInfos as $iter529)
+          {
+            $xfer += $iter529->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class WriteEventInfo {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $writeId = null;
+  /**
+   * @var string
+   */
+  public $database = null;
+  /**
+   * @var string
+   */
+  public $table = null;
+  /**
+   * @var string
+   */
+  public $files = null;
+  /**
+   * @var string
+   */
+  public $partition = null;
+  /**
+   * @var string
+   */
+  public $tableObj = null;
+  /**
+   * @var string
+   */
+  public $partitionObj = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'writeId',
+          'type' => TType::I64,
+          ),
+        2 => array(
+          'var' => 'database',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'table',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'files',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'partition',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'tableObj',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'partitionObj',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['writeId'])) {
+        $this->writeId = $vals['writeId'];
+      }
+      if (isset($vals['database'])) {
+        $this->database = $vals['database'];
+      }
+      if (isset($vals['table'])) {
+        $this->table = $vals['table'];
+      }
+      if (isset($vals['files'])) {
+        $this->files = $vals['files'];
+      }
+      if (isset($vals['partition'])) {
+        $this->partition = $vals['partition'];
+      }
+      if (isset($vals['tableObj'])) {
+        $this->tableObj = $vals['tableObj'];
+      }
+      if (isset($vals['partitionObj'])) {
+        $this->partitionObj = $vals['partitionObj'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'WriteEventInfo';
+  }
+
+  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::I64) {
+            $xfer += $input->readI64($this->writeId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->database);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->table);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->files);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->partition);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableObj);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->partitionObj);
+          } 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('WriteEventInfo');
+    if ($this->writeId !== null) {
+      $xfer += $output->writeFieldBegin('writeId', TType::I64, 1);
+      $xfer += $output->writeI64($this->writeId);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->database !== null) {
+      $xfer += $output->writeFieldBegin('database', TType::STRING, 2);
+      $xfer += $output->writeString($this->database);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->table !== null) {
+      $xfer += $output->writeFieldBegin('table', TType::STRING, 3);
+      $xfer += $output->writeString($this->table);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->files !== null) {
+      $xfer += $output->writeFieldBegin('files', TType::STRING, 4);
+      $xfer += $output->writeString($this->files);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partition !== null) {
+      $xfer += $output->writeFieldBegin('partition', TType::STRING, 5);
+      $xfer += $output->writeString($this->partition);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableObj !== null) {
+      $xfer += $output->writeFieldBegin('tableObj', TType::STRING, 6);
+      $xfer += $output->writeString($this->tableObj);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partitionObj !== null) {
+      $xfer += $output->writeFieldBegin('partitionObj', TType::STRING, 7);
+      $xfer += $output->writeString($this->partitionObj);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -16785,14 +17049,14 @@ class ReplTblWriteIdStateRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partNames = 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->partNames []= $elem528;
+              $elem535 = null;
+              $xfer += $input->readString($elem535);
+              $this->partNames []= $elem535;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16845,9 +17109,9 @@ class ReplTblWriteIdStateRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter529)
+          foreach ($this->partNames as $iter536)
           {
-            $xfer += $output->writeString($iter529);
+            $xfer += $output->writeString($iter536);
           }
         }
         $output->writeListEnd();
@@ -16922,14 +17186,14 @@ class GetValidWriteIdsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fullTableNames = 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->fullTableNames []= $elem535;
+              $elem542 = null;
+              $xfer += $input->readString($elem542);
+              $this->fullTableNames []= $elem542;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16964,9 +17228,9 @@ class GetValidWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->fullTableNames));
         {
-          foreach ($this->fullTableNames as $iter536)
+          foreach ($this->fullTableNames as $iter543)
           {
-            $xfer += $output->writeString($iter536);
+            $xfer += $output->writeString($iter543);
           }
         }
         $output->writeListEnd();
@@ -17093,14 +17357,14 @@ class TableValidWriteIds {
         case 3:
           if ($ftype == TType::LST) {
             $this->invalidWriteIds = 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->readI64($elem542);
-              $this->invalidWriteIds []= $elem542;
+              $elem549 = null;
+              $xfer += $input->readI64($elem549);
+              $this->invalidWriteIds []= $elem549;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17152,9 +17416,9 @@ class TableValidWriteIds {
       {
         $output->writeListBegin(TType::I64, count($this->invalidWriteIds));
         {
-          foreach ($this->invalidWriteIds as $iter543)
+          foreach ($this->invalidWriteIds as $iter550)
           {
-            $xfer += $output->writeI64($iter543);
+            $xfer += $output->writeI64($iter550);
           }
         }
         $output->writeListEnd();
@@ -17229,15 +17493,15 @@ class GetValidWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->tblValidWriteIds = 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\TableValidWriteIds();
-              $xfer += $elem549->read($input);
-              $this->tblValidWriteIds []= $elem549;
+              $elem556 = null;
+              $elem556 = new \metastore\TableValidWriteIds();
+              $xfer += $elem556->read($input);
+              $this->tblValidWriteIds []= $elem556;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17265,9 +17529,9 @@ class GetValidWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tblValidWriteIds));
         {
-          foreach ($this->tblValidWriteIds as $iter550)
+          foreach ($this->tblValidWriteIds as $iter557)
           {
-            $xfer += $iter550->write($output);
+            $xfer += $iter557->write($output);
           }
         }
         $output->writeListEnd();
@@ -17394,14 +17658,14 @@ class AllocateTableWriteIdsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->txnIds = 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->readI64($elem556);
-              $this->txnIds []= $elem556;
+              $elem563 = null;
+              $xfer += $input->readI64($elem563);
+              $this->txnIds []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17418,15 +17682,15 @@ class AllocateTableWriteIdsRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->srcTxnToWriteIdList = array();
-            $_size557 = 0;
-            $_etype560 = 0;
-            $xfer += $input->readListBegin($_etype560, $_size557);
-            for ($_i561 = 0; $_i561 < $_size557; ++$_i561)
+            $_size564 = 0;
+            $_etype567 = 0;
+            $xfer += $input->readListBegin($_etype567, $_size564);
+            for ($_i568 = 0; $_i568 < $_size564; ++$_i568)
             {
-              $elem562 = null;
-              $elem562 = new \metastore\TxnToWriteId();
-              $xfer += $elem562->read($input);
-              $this->srcTxnToWriteIdList []= $elem562;
+              $elem569 = null;
+              $elem569 = new \metastore\TxnToWriteId();
+              $xfer += $elem569->read($input);
+              $this->srcTxnToWriteIdList []= $elem569;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17464,9 +17728,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txnIds));
         {
-          foreach ($this->txnIds as $iter563)
+          foreach ($this->txnIds as $iter570)
           {
-            $xfer += $output->writeI64($iter563);
+            $xfer += $output->writeI64($iter570);
           }
         }
         $output->writeListEnd();
@@ -17486,9 +17750,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->srcTxnToWriteIdList));
         {
-          foreach ($this->srcTxnToWriteIdList as $iter564)
+          foreach ($this->srcTxnToWriteIdList as $iter571)
           {
-            $xfer += $iter564->write($output);
+            $xfer += $iter571->write($output);
           }
         }
         $output->writeListEnd();
@@ -17651,16 +17915,16 @@ class AllocateTableWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnToWriteIds = 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\TxnToWriteId();
-              $xfer += $elem570->read($input);
-              $this->txnToWriteIds []= $elem570;
-            }
+              $elem577 = null;
+              $elem577 = new \metastore\TxnToWriteId();
+              $xfer += $elem577->read($input);
+              $this->txnToWriteIds []= $elem577;
+            }
             $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
@@ -17687,9 +17951,9 @@ class AllocateTableWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->txnToWriteIds));
         {
-          foreach ($this->txnToWriteIds as $iter571)
+          foreach ($this->txnToWriteIds as $iter578)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter578->write($output);
           }
         }
         $output->writeListEnd();
@@ -18034,15 +18298,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = 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;
-              $elem577 = new \metastore\LockComponent();
-              $xfer += $elem577->read($input);
-              $this->component []= $elem577;
+              $elem584 = null;
+              $elem584 = new \metastore\LockComponent();
+              $xfer += $elem584->read($input);
+              $this->component []= $elem584;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18098,9 +18362,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter578)
+          foreach ($this->component as $iter585)
           {
-            $xfer += $iter578->write($output);
+            $xfer += $iter585->write($output);
           }
         }
         $output->writeListEnd();
@@ -19043,15 +19307,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = 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;
-              $elem584 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem584->read($input);
-              $this->locks []= $elem584;
+              $elem591 = null;
+              $elem591 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem591->read($input);
+              $this->locks []= $elem591;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19079,9 +19343,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter585)
+          foreach ($this->locks as $iter592)
           {
-            $xfer += $iter585->write($output);
+            $xfer += $iter592->write($output);
           }
         }
         $output->writeListEnd();
@@ -19356,17 +19620,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readSetBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size593 = 0;
+            $_etype596 = 0;
+            $xfer += $input->readSetBegin($_etype596, $_size593);
+            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
             {
-              $elem591 = null;
-              $xfer += $input->readI64($elem591);
-              if (is_scalar($elem591)) {
-                $this->aborted[$elem591] = true;
+              $elem598 = null;
+              $xfer += $input->readI64($elem598);
+              if (is_scalar($elem598)) {
+                $this->aborted[$elem598] = true;
               } else {
-                $this->aborted []= $elem591;
+                $this->aborted []= $elem598;
               }
             }
             $xfer += $input->readSetEnd();
@@ -19377,17 +19641,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size592 = 0;
-            $_etype595 = 0;
-            $xfer += $input->readSetBegin($_etype595, $_size592);
-            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
+            $_size599 = 0;
+            $_etype602 = 0;
+            $xfer += $input->readSetBegin($_etype602, $_size599);
+            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
             {
-              $elem597 = null;
-              $xfer += $input->readI64($elem597);
-              if (is_scalar($elem597)) {
-                $this->nosuch[$elem597] = true;
+              $elem604 = null;
+              $xfer += $input->readI64($elem604);
+              if (is_scalar($elem604)) {
+                $this->nosuch[$elem604] = true;
               } else {
-                $this->nosuch []= $elem597;
+                $this->nosuch []= $elem604;
               }
             }
             $xfer += $input->readSetEnd();
@@ -19416,12 +19680,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter598 => $iter599)
+          foreach ($this->aborted as $iter605 => $iter606)
           {
-            if (is_scalar($iter599)) {
-            $xfer += $output->writeI64($iter598);
+            if (is_scalar($iter606)) {
+            $xfer += $output->writeI64($iter605);
             } else {
-            $xfer += $output->writeI64($iter599);
+            $xfer += $output->writeI64($iter606);
             }
           }
         }
@@ -19437,12 +19701,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter600 => $iter601)
+          foreach ($this->nosuch as $iter607 => $iter608)
           {
-            if (is_scalar($iter601)) {
-            $xfer += $output->writeI64($iter600);
+            if (is_scalar($iter608)) {
+            $xfer += $output->writeI64($iter607);
             } else {
-            $xfer += $output->writeI64($iter601);
+            $xfer += $output->writeI64($iter608);
             }
           }
         }
@@ -19601,17 +19865,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size602 = 0;
-            $_ktype603 = 0;
-            $_vtype604 = 0;
-            $xfer += $input->readMapBegin($_ktype603, $_vtype604, $_size602);
-            for ($_i606 = 0; $_i606 < $_size602; ++$_i606)
+            $_size609 = 0;
+            $_ktype610 = 0;
+            $_vtype611 = 0;
+            $xfer += $input->readMapBegin($_ktype610, $_vtype611, $_size609);
+            for ($_i613 = 0; $_i613 < $_size609; ++$_i613)
             {
-              $key607 = '';
-              $val608 = '';
-              $xfer += $input->readString($key607);
-              $xfer += $input->readString($val608);
-              $this->properties[$key607] = $val608;
+              $key614 = '';
+              $val615 = '';
+              $xfer += $input->readString($key614);
+              $xfer += $input->readString($val615);
+              $this->properties[$key614] = $val615;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19664,10 +19928,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter609 => $viter610)
+          foreach ($this->properties as $kiter616 => $viter617)
           {
-            $xfer += $output->writeString($kiter609);
-            $xfer += $output->writeString($viter610);
+            $xfer += $output->writeString($kiter616);
+            $xfer += $output->writeString($viter617);
           }
         }
         $output->writeMapEnd();
@@ -20254,15 +20518,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem616->read($input);
-              $this->compacts []= $elem616;
+              $elem623 = null;
+              $elem623 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem623->read($input);
+              $this->compacts []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20290,9 +20554,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter617)
+          foreach ($this->compacts as $iter624)
           {
-            $xfer += $iter617->write($output);
+            $xfer += $iter624->write($output);
           }
         }
         $output->writeListEnd();
@@ -20439,14 +20703,14 @@ class AddDynamicPartitions {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size618 = 0;
-            $_etype621 = 0;
-            $xfer += $input->readListBegin($_etype621, $_size618);
-            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem623 = null;
-              $xfer += $input->readString($elem623);
-              $this->partitionnames []= $elem623;
+              $elem630 = null;
+              $xfer += $input->readString($elem630);
+              $this->partitionnames []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20501,9 +20765,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter624)
+          foreach ($this->partitionnames as $iter631)
           {
-            $xfer += $output->writeString($iter624);
+            $xfer += $output->writeString($iter631);
           }
         }
         $output->writeListEnd();
@@ -20827,17 +21091,17 @@ class CreationMetadata {
         case 4:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readSetBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readSetBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem630 = null;
-              $xfer += $input->readString($elem630);
-              if (is_scalar($elem630)) {
-                $this->tablesUsed[$elem630] = true;
+              $elem637 = null;
+              $xfer += $input->readString($elem637);
+              if (is_scalar($elem637)) {
+                $this->tablesUsed[$elem637] = true;
               } else {
-                $this->tablesUsed []= $elem630;
+                $this->tablesUsed []= $elem637;
               }
             }
             $xfer += $input->readSetEnd();
@@ -20888,12 +21152,12 @@ class CreationMetadata {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter631 => $iter632)
+          foreach ($this->tablesUsed as $iter638 => $iter639)
           {
-            if (is_scalar($iter632)) {
-            $xfer += $output->writeString($iter631);
+            if (is_scalar($iter639)) {
+            $xfer += $output->writeString($iter638);
             } else {
-            $xfer += $output->writeString($iter632);
+            $xfer += $output->writeString($iter639);
             }
           }
         }
@@ -21298,15 +21562,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size633 = 0;
-            $_etype636 = 0;
-            $xfer += $input->readListBegin($_etype636, $_size633);
-            for ($_i637 = 0; $_i637 < $_size633; ++$_i637)
+            $_size640 = 0;
+            $_etype643 = 0;
+            $xfer += $input->readListBegin($_etype643, $_size640);
+            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
             {
-              $elem638 = null;
-              $elem638 = new \metastore\NotificationEvent();
-              $xfer += $elem638->read($input);
-              $this->events []= $elem638;
+              $elem645 = null;
+              $elem645 = new \metastore\NotificationEvent();
+              $xfer += $elem645->read($input);
+              $this->events []= $elem645;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21334,9 +21598,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter639)
+          foreach ($this->events as $iter646)
           {
-            $xfer += $iter639->write($output);
+            $xfer += $iter646->write($output);
           }
         }
         $output->writeListEnd();
@@ -21636,6 +21900,10 @@ class InsertEventRequestData {
    * @var string[]
    */
   public $filesAddedChecksum = null;
+  /**
+   * @var string[]
+   */
+  public $subDirectoryList = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -21660,6 +21928,14 @@ class InsertEventRequestData {
             'type' => TType::STRING,
             ),
           ),
+        4 => array(
+          'var' => 'subDirectoryList',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -21672,6 +21948,9 @@ class InsertEventRequestData {
       if (isset($vals['filesAddedChecksum'])) {
         $this->filesAddedChecksum = $vals['filesAddedChecksum'];
       }
+      if (isset($vals['subDirectoryList'])) {
+        $this->subDirectoryList = $vals['subDirectoryList'];
+      }
     }
   }
 
@@ -21704,14 +21983,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size640 = 0;
-            $_etype643 = 0;
-            $xfer += $input->readListBegin($_etype643, $_size640);
-            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
+            $_size647 = 0;
+            $_etype650 = 0;
+            $xfer += $input->readListBegin($_etype650, $_size647);
+            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
             {
-              $elem645 = null;
-              $xfer += $input->readString($elem645);
-              $this->filesAdded []= $elem645;
+              $elem652 = null;
+              $xfer += $input->readString($elem652);
+              $this->filesAdded []= $elem652;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21721,14 +22000,31 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = 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)
+            {
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->filesAddedChecksum []= $elem658;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::LST) {
+            $this->subDirectoryList = array();
+            $_size659 = 0;
+            $_etype662 = 0;
+            $xfer += $input->readListBegin($_etype662, $_size659);
+            for ($_i663 = 0; $_i663 < $_size659; ++$_i663)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->filesAddedChecksum []= $elem651;
+              $elem664 = null;
+              $xfer += $input->readString($elem664);
+              $this->subDirectoryList []= $elem664;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21761,9 +22057,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter652)
+          foreach ($this->filesAdded as $iter665)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter665);
           }
         }
         $output->writeListEnd();
@@ -21778,9 +22074,26 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter653)
+          foreach ($this->filesAddedChecksum as $iter666)
+          {
+            $xfer += $output->writeString($iter666);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->subDirectoryList !== null) {
+      if (!is_array($this->subDirectoryList)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('subDirectoryList', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRING, count($this->subDirectoryList));
+        {
+          foreach ($this->subDirectoryList as $iter667)
           {
-            $xfer += $output->writeString($iter653);
+            $xfer += $output->writeString($iter667);
           }
         }
         $output->writeListEnd();
@@ -21794,33 +22107,320 @@ class InsertEventRequestData {
 
 }
 
-class FireEventRequestData {
+class FireEventRequestData {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\InsertEventRequestData
+   */
+  public $insertData = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'insertData',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InsertEventRequestData',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['insertData'])) {
+        $this->insertData = $vals['insertData'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FireEventRequestData';
+  }
+
+  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->insertData = new \metastore\InsertEventRequestData();
+            $xfer += $this->insertData->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('FireEventRequestData');
+    if ($this->insertData !== null) {
+      if (!is_object($this->insertData)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('insertData', TType::STRUCT, 1);
+      $xfer += $this->insertData->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FireEventRequest {
+  static $_TSPEC;
+
+  /**
+   * @var bool
+   */
+  public $successful = null;
+  /**
+   * @var \metastore\FireEventRequestData
+   */
+  public $data = null;
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
+   * @var string[]
+   */
+  public $partitionVals = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'successful',
+          'type' => TType::BOOL,
+          ),
+        2 => array(
+          'var' => 'data',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\FireEventRequestData',
+          ),
+        3 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'partitionVals',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        6 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['successful'])) {
+        $this->successful = $vals['successful'];
+      }
+      if (isset($vals['data'])) {
+        $this->data = $vals['data'];
+      }
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tableName'])) {
+        $this->tableName = $vals['tableName'];
+      }
+      if (isset($vals['partitionVals'])) {
+        $this->partitionVals = $vals['partitionVals'];
+      }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FireEventRequest';
+  }
+
+  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::BOOL) {
+            $xfer += $input->readBool($this->successful);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->data = new \metastore\FireEventRequestData();
+            $xfer += $this->data->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::LST) {
+            $this->partitionVals = array();
+            $_size668 = 0;
+            $_etype671 = 0;
+            $xfer += $input->readListBegin($_etype671, $_size668);
+            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
+            {
+              $elem673 = null;
+              $xfer += $input->readString($elem673);
+              $this->partitionVals []= $elem673;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } 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('FireEventRequest');
+    if ($this->successful !== null) {
+      $xfer += $output->writeFieldBegin('successful', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->successful);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->data !== null) {
+      if (!is_object($this->data)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('data', TType::STRUCT, 2);
+      $xfer += $this->data->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 4);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partitionVals !== null) {
+      if (!is_array($this->partitionVals)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 5);
+      {
+        $output->writeListBegin(TType::STRING, count($this->partitionVals));
+        {
+          foreach ($this->partitionVals as $iter674)
+          {
+            $xfer += $output->writeString($iter674);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FireEventResponse {
   static $_TSPEC;
 
-  /**
-   * @var \metastore\InsertEventRequestData
-   */
-  public $insertData = null;
 
-  public function __construct($vals=null) {
+  public function __construct() {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
-        1 => array(
-          'var' => 'insertData',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\InsertEventRequestData',
-          ),
         );
     }
-    if (is_array($vals)) {
-      if (isset($vals['insertData'])) {
-        $this->insertData = $vals['insertData'];
-      }
-    }
   }
 
   public function getName() {
-    return 'FireEventRequestData';
+    return 'FireEventResponse';
   }
 
   public function read($input)
@@ -21838,14 +22438,6 @@ class FireEventRequestData {
       }
       switch ($fid)
       {
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->insertData = new \metastore\InsertEventRequestData();
-            $xfer += $this->insertData->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -21858,15 +22450,7 @@ class FireEventRequestData {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventRequestData');
-    if ($this->insertData !== null) {
-      if (!is_object($this->insertData)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('insertData', TType::STRUCT, 1);
-      $xfer += $this->insertData->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
+    $xfer += $output->writeStructBegin('FireEventResponse');
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -21874,55 +22458,59 @@ class FireEventRequestData {
 
 }
 
-class FireEventRequest {
+class WriteNotificationLogRequest {
   static $_TSPEC;
 
   /**
-   * @var bool
+   * @var int
    */
-  public $successful = null;
+  public $txnId = null;
   /**
-   * @var \metastore\FireEventRequestData
+   * @var int
    */
-  public $data = null;
+  public $writeId = null;
   /**
    * @var string
    */
-  public $dbName = null;
+  public $db = null;
   /**
    * @var string
    */
-  public $tableName = null;
+  public $table = null;
   /**
-   * @var string[]
+   * @var \metastore\InsertEventRequestData
    */
-  public $partitionVals = null;
+  public $fileInfo = null;
   /**
-   * @var string
+   * @var string[]
    */
-  public $catName = null;
+  public $partitionVals = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'successful',
-          'type' => TType::BOOL,
+          'var' => 'txnId',
+          'type' => TType::I64,
           ),
         2 => array(
-          'var' => 'data',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\FireEventRequestData',
+          'var' => 'writeId',
+          'type' => TType::I64,
           ),
         3 => array(
-          'var' => 'dbName',
+          'var' => 'db',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'tableName',
+          'var' => 'table',
           'type' => TType::STRING,
           ),
         5 => array(
+          'var' => 'fileInfo',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InsertEventRequestData',
+          ),
+        6 => array(
           'var' => 'partitionVals',
           'type' => TType::LST,
           'etype' => TType::STRING,
@@ -21930,36 +22518,32 @@ class FireEventRequest {
             'type' => TType::STRING,
             ),
           ),
-        6 => array(
-          'var' => 'catName',
-          'type' => TType::STRING,
-          ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['successful'])) {
-        $this->successful = $vals['successful'];
+      if (isset($vals['txnId'])) {
+        $this->txnId = $vals['txnId'];
       }
-      if (isset($vals['data'])) {
-        $this->data = $vals['data'];
+      if (isset($vals['writeId'])) {
+        $this->writeId = $vals['writeId'];
       }
-      if (isset($vals['dbName'])) {
-        $this->dbName = $vals['dbName'];
+      if (isset($vals['db'])) {
+        $this->db = $vals['db'];
       }
-      if (isset($vals['tableName'])) {
-        $this->tableName = $vals['tableName'];
+      if (isset($vals['table'])) {
+        $this->table = $vals['table'];
+      }
+      if (isset($vals['fileInfo'])) {
+        $this->fileInfo = $vals['fileInfo'];
       }
       if (isset($vals['partitionVals'])) {
         $this->partitionVals = $vals['partitionVals'];
       }
-      if (isset($vals['catName'])) {
-        $this->catName = $vals['catName'];
-      }
     }
   }
 
   public function getName() {
-    return 'FireEventRequest';
+    return 'WriteNotificationLogRequest';
   }
 
   public function read($input)
@@ -21978,54 +22562,54 @@ class FireEventRequest {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->successful);
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->txnId);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::STRUCT) {
-            $this->data = new \metastore\FireEventRequestData();
-            $xfer += $this->data->read($input);
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->writeId);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbName);
+            $xfer += $input->readString($this->db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->tableName);
+            $xfer += $input->readString($this->table);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
-          if ($ftype == TType::LST) {
-            $this->partitionVals = array();
-            $_size654 = 0;
-            $_etype657 = 0;
-            $xfer += $input->readListBegin($_etype657, $_size654);
-            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
-            {
-              $elem659 = null;
-              $xfer += $input->readString($elem659);
-              $this->partitionVals []= $elem659;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->fileInfo = new \metastore\InsertEventRequestData();
+            $xfer += $this->fileInfo->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 6:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->catName);
+          if ($ftype == TType::LST) {
+            $this->partitionVals = array();
+            $_size675 = 0;
+            $_etype678 = 0;
+            $xfer += $input->readListBegin($_etype678, $_size675);
+            for ($_i679 = 0; $_i679 < $_size675; ++$_i679)
+            {
+              $elem680 = null;
+              $xfer += $input->readString($elem680);
+              $this->partitionVals []= $elem680;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -22042,52 +22626,52 @@ class FireEventRequest {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventRequest');
-    if ($this->successful !== null) {
-      $xfer += $output->writeFieldBegin('successful', TType::BOOL, 1);
-      $xfer += $output->writeBool($this->successful);
+    $xfer += $output->writeStructBegin('WriteNotificationLogRequest');
+    if ($this->txnId !== null) {
+      $xfer += $output->writeFieldBegin('txnId', TType::I64, 1);
+      $xfer += $output->writeI64($this->txnId);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->data !== null) {
-      if (!is_object($this->data)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('data', TType::STRUCT, 2);
-      $xfer += $this->data->write($output);
+    if ($this->writeId !== null) {
+      $xfer += $output->writeFieldBegin('writeId', TType::I64, 2);
+      $xfer += $output->writeI64($this->writeId);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
-      $xfer += $output->writeString($this->dbName);
+    if ($this->db !== null) {
+      $xfer += $output->writeFieldBegin('db', TType::STRING, 3);
+      $xfer += $output->writeString($this->db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->tableName !== null) {
-      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 4);
-      $xfer += $output->writeString($this->tableName);
+    if ($this->table !== null) {
+      $xfer += $output->writeFieldBegin('table', TType::STRING, 4);
+      $xfer += $output->writeString($this->table);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->fileInfo !== null) {
+      if (!is_object($this->fileInfo)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileInfo', TType::STRUCT, 5);
+      $xfer += $this->fileInfo->write($output);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->partitionVals !== null) {
       if (!is_array($this->partitionVals)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 5);
+      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 6);
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter660)
+          foreach ($this->partitionVals as $iter681)
           {
-            $xfer += $output->writeString($iter660);
+            $xfer += $output->writeString($iter681);
           }
         }
         $output->writeListEnd();
       }
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->catName !== null) {
-      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
-      $xfer += $output->writeString($this->catName);
-      $xfer += $output->writeFieldEnd();
-    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -22095,7 +22679,7 @@ class FireEventRequest {
 
 }
 
-class FireEventResponse {
+class WriteNotificationLogResponse {
   static $_TSPEC;
 
 
@@ -22107,7 +22691,7 @@ class FireEventResponse {
   }
 
   public function getName() {
-    return 'FireEventResponse';
+    return 'WriteNotificationLogResponse';
   }
 
   public function read($input)
@@ -22137,7 +22721,7 @@ class FireEventResponse {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventResponse');
+    $xfer += $output->writeStructBegin('WriteNotificationLogResponse');
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -22309,18 +22893,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size661 = 0;
-            $_ktype662 = 0;
-            $_vtype663 = 0;
-            $xfer += $input->readMapBegin($_ktype662, $_vtype663, $_size661);
-            for ($_i665 = 0; $_i665 < $_size661; ++$_i665)
+            $_size682 = 0;
+            $_ktype683 = 0;
+            $_vtype684 = 0;
+            $xfer += $input->readMapBegin($_ktype683, $_vtype684, $_size682);
+            for ($_i686 = 0; $_i686 < $_size682; ++$_i686)
             {
-              $key666 = 0;
-              $val667 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key666);
-              $val667 = new \metastore\MetadataPpdResult();
-              $xfer += $val667->read($input);
-              $this->metadata[$key666] = $val667;
+              $key687 = 0;
+              $val688 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key687);
+              $val688 = new \metastore\MetadataPpdResult();
+              $xfer += $val688->read($input);
+              $this->metadata[$key687] = $val688;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22355,10 +22939,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter668 => $viter669)
+          foreach ($this->metadata as $kiter689 => $viter690)
           {
-            $xfer += $output->writeI64($kiter668);
-            $xfer += $viter669->write($output);
+            $xfer += $output->writeI64($kiter689);
+            $xfer += $viter690->write($output);
           }
         }
         $output->writeMapEnd();
@@ -22460,14 +23044,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size670 = 0;
-            $_etype673 = 0;
-            $xfer += $input->readListBegin($_etype673, $_size670);
-            for ($_i674 = 0; $_i674 < $_size670; ++$_i674)
+            $_size691 = 0;
+            $_etype694 = 0;
+            $xfer += $input->readListBegin($_etype694, $_size691);
+            for ($_i695 = 0; $_i695 < $_size691; ++$_i695)
             {
-              $elem675 = null;
-              $xfer += $input->readI64($elem675);
-              $this->fileIds []= $elem675;
+              $elem696 = null;
+              $xfer += $input->readI64($elem696);
+              $this->fileIds []= $elem696;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22516,9 +23100,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter676)
+          foreach ($this->fileIds as $iter697)
           {
-            $xfer += $output->writeI64($iter676);
+            $xfer += $output->writeI64($iter697);
           }
         }
         $output->writeListEnd();
@@ -22612,17 +23196,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size677 = 0;
-            $_ktype678 = 0;
-            $_vtype679 = 0;
-            $xfer += $input->readMapBegin($_ktype678, $_vtype679, $_size677);
-            for ($_i681 = 0; $_i681 < $_size677; ++$_i681)
+            $_size698 = 0;
+            $_ktype699 = 0;
+            $_vtype700 = 0;
+            $xfer += $input->readMapBegin($_ktype699, $_vtype700, $_size698);
+            for ($_i702 = 0; $_i702 < $_size698; ++$_i702)
             {
-              $key682 = 0;
-              $val683 = '';
-              $xfer += $input->readI64($key682);
-              $xfer += $input->readString($val683);
-              $this->metadata[$key682] = $val683;
+              $key703 = 0;
+              $val704 = '';
+              $xfer += $input->readI64($key703);
+              $xfer += $input->readString($val704);
+              $this->metadata[$key703] = $val704;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22657,10 +23241,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter684 => $viter685)
+          foreach ($this->metadata as $kiter705 => $viter706)
           {
-            $xfer += $output->writeI64($kiter684);
-            $xfer += $output->writeString($viter685);
+            $xfer += $output->writeI64($kiter705);
+            $xfer += $output->writeString($viter706);
           }
         }
         $output->writeMapEnd();
@@ -22729,14 +23313,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size686 = 0;
-            $_etype689 = 0;
-            $xfer += $input->readListBegin($_etype689, $_size686);
-            for ($_i690 = 0; $_i690 < $_size686; ++$_i690)
+            $_size707 = 0;
+            $_etype710 = 0;
+            $xfer += $input->readListBegin($_etype710, $_size707);
+            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
             {
-              $elem691 = null;
-              $xfer += $input->readI64($elem691);
-              $this->fileIds []= $elem691;
+              $elem712 = null;
+              $xfer += $input->readI64($elem712);
+              $this->fileIds []= $elem712;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22764,9 +23348,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter692)
+          foreach ($this->fileIds as $iter713)
           {
-            $xfer += $output->writeI64($iter692);
+            $xfer += $output->writeI64($iter713);
           }
         }
         $output->writeListEnd();
@@ -22906,14 +23490,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size693 = 0;
-            $_etype696 = 0;
-            $xfer += $input->readListBegin($_etype696, $_size693);
-            for ($_i697 = 0; $_i697 < $_size693; ++$_i697)
+            $_size714 = 0;
+            $_etype717 = 0;
+            $xfer += $input->readListBegin($_etype717, $_size714);
+            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
             {
-              $elem698 = null;
-              $xfer += $input->readI64($elem698);
-              $this->fileIds []= $elem698;
+              $elem719 = null;
+              $xfer += $input->readI64($elem719);
+              $this->fileIds []= $elem719;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22923,14 +23507,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size699 = 0;
-            $_etype702 = 0;
-            $xfer += $input->readListBegin($_etype702, $_size699);
-            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
+            $_size720 = 0;
+            $_etype723 = 0;
+            $xfer += $input->readListBegin($_etype723, $_size720);
+            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
             {
-              $elem704 = null;
-              $xfer += $input->readString($elem704);
-              $this->metadata []= $elem704;
+              $elem725 = null;
+              $xfer += $input->readString($elem725);
+              $this->metadata []= $elem725;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22965,9 +23549,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter705)
+          foreach ($this->fileIds as $iter726)
           {
-            $xfer += $output->writeI64($iter705);
+            $xfer += $output->writeI64($iter726);
           }
         }
         $output->writeListEnd();
@@ -22982,9 +23566,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter706)
+          foreach ($this->metadata as $iter727)
           {
-            $xfer += $output->writeString($iter706);
+            $xfer += $output->writeString($iter727);
           }
         }
         $output->writeListEnd();
@@ -23103,14 +23687,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size707 = 0;
-            $_etype710 = 0;
-            $xfer += $input->readListBegin($_etype710, $_size707);
-            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
+            $_size728 = 0;
+            $_etype731 = 0;
+            $xfer += $input->readListBegin($_etype731, $_size728);
+            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
             {
-              $elem712 = null;
-              $xfer += $input->readI64($elem712);
-              $this->fileIds []= $elem712;
+              $elem733 = null;
+              $xfer += $input->readI64($elem733);
+              $this->fileIds []= $elem733;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23138,9 +23722,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter713)
+          foreach ($this->fileIds as $iter734)
           {
-            $xfer += $output->writeI64($iter713);
+            $xfer += $output->writeI64($iter734);
           }
         }
         $output->writeListEnd();
@@ -23424,15 +24008,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size714 = 0;
-            $_etype717 = 0;
-            $xfer += $input->readListBegin($_etype717, $_size714);
-            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
+            $_size735 = 0;
+            $_etype738 = 0;
+            $xfer += $input->readListBegin($_etype738, $_size735);
+            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
             {
-              $elem719 = null;
-              $elem719 = new \metastore\Function();
-              $xfer += $elem719->read($input);
-              $this->functions []= $elem719;
+              $elem740 = null;
+              $elem740 = new \metastore\Function();
+              $xfer += $elem740->read($input);
+              $this->functions []= $elem740;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23460,9 +24044,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter720)
+          foreach ($this->functions as $iter741)
           {
-            $xfer += $iter720->write($output);
+            $xfer += $iter741->write($output);
           }
         }
         $output->writeListEnd();
@@ -23526,14 +24110,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size721 = 0;
-            $_etype724 = 0;
-            $xfer += $input->readListBegin($_etype724, $_size721);
-            for ($_i725 = 0; $_i725 < $_size721; ++$_i725)
+            $_size742 = 0;
+            $_etype745 = 0;
+            $xfer += $input->readListBegin($_etype745, $_size742);
+            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
             {
-              $elem726 = null;
-              $xfer += $input->readI32($elem726);
-              $this->values []= $elem726;
+              $elem747 = null;
+              $xfer += $input->readI32($elem747);
+              $this->values []= $elem747;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23561,9 +24145,9 @@ class ClientCapabilities {
       {
         $output->writeListBegin(TType::I32, count($this->values));
         {
-          foreach ($this->values as $iter727)
+          foreach ($this->values as $iter748)
           {
-            $xfer += $output->writeI32($iter727);
+            $xfer += $output->writeI32($iter748);
           }
         }
         $output->writeListEnd();
@@ -23897,14 +24481,14 @@ class GetTablesRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->tblNames = array();
-            $_size728 = 0;
-            $_etype731 = 0;
-            $xfer += $input->readListBegin($_etype731, $_size728);
-            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
+            $_size749 = 0;
+            $_etype752 = 0;
+            $xfer += $input->readListBegin($_etype752, $_size749);
+            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
             {
-              $elem733 = null;
-              $xfer += $input->readString($elem733);
-              $this->tblNames []= $elem733;
+              $elem754 = null;
+              $xfer += $input->readString($elem754);
+              $this->tblNames []= $elem754;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23952,9 +24536,9 @@ class GetTablesRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->tblNames));
         {
-          foreach ($this->tblNames as $iter734)
+          foreach ($this->tblNames as $iter755)
           {
-            $xfer += $output->writeString($iter734);
+            $xfer += $output->writeString($iter755);
           }
         }
         $output->writeListEnd();
@@ -24032,15 +24616,15 @@ class GetTablesResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tables = array();
-            $_size735 = 0;
-            $_etype738 = 0;
-            $xfer += $input->readListBegin($_etype738, $_size735);
-            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
+            $_size756 = 0;
+            $_etype759 = 0;
+            $xfer += $input->readListBegin($_etype759, $_size756);
+            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
             {
-              $elem740 = null;
-              $elem740 = new \metastore\Table();
-              $xfer += $elem740->read($input);
-              $this->tables []= $elem740;
+              $elem761 = null;
+              $elem761 = new \metastore\Table();
+              $xfer += $elem761->read($input);
+              $this->tables []= $elem761;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24068,9 +24652,9 @@ class GetTablesResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tables));
         {
-          foreach ($this->tables as $iter741)
+          foreach ($this->tables as $iter762)
           {
-            $xfer += $iter741->write($output);
+            $xfer += $iter762->write($output);
           }
         }
         $output->writeListEnd();
@@ -24482,17 +25066,17 @@ class Materialization {
         case 1:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size742 = 0;
-            $_etype745 = 0;
-            $xfer += $input->readSetBegin($_etype745, $_size742);
-            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
+            $_size763 = 0;
+            $_etype766 = 0;
+            $xfer += $input->readSetBegin($_etype766, $_size763);
+            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
             {
-              $elem747 = null;
-              $xfer += $input->readString($elem747);
-              if (is_scalar($elem747)) {
-                $this->tablesUsed[$elem747] = true;
+              $elem768 = null;
+              $xfer += $input->readString($elem768);
+              if (is_scalar($elem768)) {
+                $this->tablesUsed[$elem768] = true;
               } else {
-                $this->tablesUsed []= $elem747;
+                $this->tablesUsed []= $elem768;
               }
             }
             $xfer += $input->readSetEnd();
@@ -24542,12 +25126,12 @@ class Materialization {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter748 => $iter749)
+          foreach ($this->tablesUsed as $iter769 => $iter770)
           {
-            if (is_scalar($iter749)) {
-            $xfer += $output->writeString($iter748);
+            if (is_scalar($iter770)) {
+            $xfer += $output->writeString($iter769);
             } else {
-            $xfer += $output->writeString($iter749);
+            $xfer += $output->writeString($iter770);
             }
           }
         }
@@ -25819,15 +26403,15 @@ class WMFullResourcePlan {
         case 2:
           if ($ftype == TType::LST) {
             $this->pools = array();
-            $_size750 = 0;
-            $_etype753 = 0;
-            $xfer += $input->readListBegin($_etype753, $_size750);
-            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
+            $_size771 = 0;
+            $_etype774 = 0;
+            $xfer += $input->readListBegin($_etype774, $_size771);
+            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
             {
-              $elem755 = null;
-              $elem755 = new \metastore\WMPool();
-              $xfer += $elem755->read($input);
-              $this->pools []= $elem755;
+              $elem776 = null;
+              $elem776 = new \metastore\WMPool();
+              $xfer += $elem776->read($input);
+              $this->pools []= $elem776;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25837,15 +26421,15 @@ class WMFullResourcePlan {
         case 3:
           if ($ftype == TType::LST) {
             $this->mappings = array();
-            $_size756 = 0;
-            $_etype759 = 0;
-            $xfer += $input->readListBegin($_etype759, $_size756);
-            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
+            $_size777 = 0;
+            $_etype780 = 0;
+            $xfer += $input->readListBegin($_etype780, $_size777);
+            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
             {
-              $elem761 = null;
-              $elem761 = new \metastore\WMMapping();
-              $xfer += $elem761->read($input);
-              $this->mappings []= $elem761;
+              $elem782 = null;
+              $elem782 = new \metastore\WMMapping();
+              $xfer += $elem782->read($input);
+              $this->mappings []= $elem782;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25855,15 +26439,15 @@ class WMFullResourcePlan {
         case 4:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\WMTrigger();
-              $xfer += $elem767->read($input);
-              $this->triggers []= $elem767;
+              $elem788 = null;
+              $elem788 = new \metastore\WMTrigger();
+              $xfer += $elem788->read($input);
+              $this->triggers []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25873,15 +26457,15 @@ class WMFullResourcePlan {
         case 5:
           if ($ftype == TType::LST) {
             $this->poolTriggers = array();
-            $_size768 = 0;
-            $_etype771 = 0;
-            $xfer += $input->readListBegin($_etype771, $_size768);
-            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
+            $_size789 = 0;
+            $_etype792 = 0;
+            $xfer += $input->readListBegin($_etype792, $_size789);
+            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
             {
-              $elem773 = null;
-              $elem773 = new \metastore\WMPoolTrigger();
-              $xfer += $elem773->read($input);
-              $this->poolTriggers []= $elem773;
+              $elem794 = null;
+              $elem794 = new \metastore\WMPoolTrigger();
+              $xfer += $elem794->read($input);
+              $this->poolTriggers []= $elem794;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25917,9 +26501,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->pools));
         {
-          foreach ($this->pools as $iter774)
+          foreach ($this->pools as $iter795)
           {
-            $xfer += $iter774->write($output);
+            $xfer += $iter795->write($output);
           }
         }
         $output->writeListEnd();
@@ -25934,9 +26518,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->mappings));
         {
-          foreach ($this->mappings as $iter775)
+          foreach ($this->mappings as $iter796)
           {
-            $xfer += $iter775->write($output);
+            $xfer += $iter796->write($output);
           }
         }
         $output->writeListEnd();
@@ -25951,9 +26535,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter776)
+          foreach ($this->triggers as $iter797)
           {
-            $xfer += $iter776->write($output);
+            $xfer += $iter797->write($output);
           }
         }
         $output->writeListEnd();
@@ -25968,9 +26552,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->poolTriggers));
         {
-          foreach ($this->poolTriggers as $iter777)
+          foreach ($this->poolTriggers as $iter798)
           {
-            $xfer += $iter777->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -26523,15 +27107,15 @@ class WMGetAllResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->resourcePlans = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem783 = null;
-              $elem783 = new \metastore\WMResourcePlan();
-              $xfer += $elem783->read($input);
-              $this->resourcePlans []= $elem783;
+              $elem804 = null;
+              $elem804 = new \metastore\WMResourcePlan();
+              $xfer += $elem804->read($input);
+              $this->resourcePlans []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26559,9 +27143,9 @@ class WMGetAllResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourcePlans));
         {
-          foreach ($this->resourcePlans as $iter784)
+          foreach ($this->resourcePlans as $iter805)
           {
-            $xfer += $iter784->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -26967,14 +27551,14 @@ class WMValidateResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->errors = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem790 = null;
-              $xfer += $input->readString($elem790);
-              $this->errors []= $elem790;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->errors []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26984,14 +27568,14 @@ class WMValidateResourcePlanResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->warnings = array();
-            $_size791 = 0;
-            $_etype794 = 0;
-            $xfer += $input->readListBegin($_etype794, $_size791);
-            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
+            $_size812 = 0;
+            $_etype815 = 0;
+            $xfer += $input->readListBegin($_etype815, $_size812);
+            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
             {
-              $elem796 = null;
-              $xfer += $input->readString($elem796);
-              $this->warnings []= $elem796;
+              $elem817 = null;
+              $xfer += $input->readString($elem817);
+              $this->warnings []= $elem817;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27019,9 +27603,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter797)
+          foreach ($this->errors as $iter818)
           {
-            $xfer += $output->writeString($iter797);
+            $xfer += $output->writeString($iter818);
           }
         }
         $output->writeListEnd();
@@ -27036,9 +27620,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->warnings));
         {
-          foreach ($this->warnings as $iter798)
+          foreach ($this->warnings as $iter819)
           {
-            $xfer += $output->writeString($iter798);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -27711,15 +28295,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem804 = null;
-              $elem804 = new \metastore\WMTrigger();
-              $xfer += $elem804->read($input);
-              $this->triggers []= $elem804;
+              $elem825 = null;
+              $elem825 = new \metastore\WMTrigger();
+              $xfer += $elem825->read($input);
+              $this->triggers []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27747,9 +28331,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter805)
+          foreach ($this->triggers as $iter826)
           {
-            $xfer += $iter805->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -29333,15 +29917,15 @@ class SchemaVersion {
         case 4:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem811 = null;
-              $elem811 = new \metastore\FieldSchema();
-              $xfer += $elem811->read($input);
-              $this->cols []= $elem811;
+              $elem832 = null;
+              $elem832 = new \metastore\FieldSchema();
+              $xfer += $elem832->read($input);
+              $this->cols []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29430,9 +30014,9 @@ class SchemaVersion {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter812)
+          foreach ($this->cols as $iter833)
           {
-            $xfer += $iter812->write($output);
+            $xfer += $iter833->write($output);
           }
         }
         $output->writeListEnd();
@@ -29754,15 +30338,15 @@ class FindSchemasByColsResp {
         case 1:
           if ($ftype == TType::LST) {
             $this->schemaVersions = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size834 = 0;
+            $_etype837 = 0;
+            $xfer += $input->readListBegin($_etype837, $_size834);
+            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\SchemaVersionDescriptor();
-              $xfer += $elem818->read($input);
-              $this->schemaVersions []= $elem818;
+              $elem839 = null;
+              $elem839 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem839->read($input);
+              $this->schemaVersions []= $elem839;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29790,9 +30374,9 @@ class FindSchemasByColsResp {
       {
         $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
         {
-          foreach ($this->schemaVersions as $iter819)
+          foreach ($this->schemaVersions as $iter840)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter840->write($output);
           }
         }
         $output->writeListEnd();


[03/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 11affe3..031e72b 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -11562,17 +11562,20 @@ class CommitTxnRequest:
   Attributes:
    - txnid
    - replPolicy
+   - writeEventInfos
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.I64, 'txnid', None, None, ), # 1
     (2, TType.STRING, 'replPolicy', None, None, ), # 2
+    (3, TType.LIST, 'writeEventInfos', (TType.STRUCT,(WriteEventInfo, WriteEventInfo.thrift_spec)), None, ), # 3
   )
 
-  def __init__(self, txnid=None, replPolicy=None,):
+  def __init__(self, txnid=None, replPolicy=None, writeEventInfos=None,):
     self.txnid = txnid
     self.replPolicy = replPolicy
+    self.writeEventInfos = writeEventInfos
 
   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:
@@ -11593,6 +11596,17 @@ class CommitTxnRequest:
           self.replPolicy = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.writeEventInfos = []
+          (_etype526, _size523) = iprot.readListBegin()
+          for _i527 in xrange(_size523):
+            _elem528 = WriteEventInfo()
+            _elem528.read(iprot)
+            self.writeEventInfos.append(_elem528)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -11611,6 +11625,13 @@ class CommitTxnRequest:
       oprot.writeFieldBegin('replPolicy', TType.STRING, 2)
       oprot.writeString(self.replPolicy)
       oprot.writeFieldEnd()
+    if self.writeEventInfos is not None:
+      oprot.writeFieldBegin('writeEventInfos', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRUCT, len(self.writeEventInfos))
+      for iter529 in self.writeEventInfos:
+        iter529.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -11624,6 +11645,158 @@ class CommitTxnRequest:
     value = 17
     value = (value * 31) ^ hash(self.txnid)
     value = (value * 31) ^ hash(self.replPolicy)
+    value = (value * 31) ^ hash(self.writeEventInfos)
+    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 WriteEventInfo:
+  """
+  Attributes:
+   - writeId
+   - database
+   - table
+   - files
+   - partition
+   - tableObj
+   - partitionObj
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'writeId', None, None, ), # 1
+    (2, TType.STRING, 'database', None, None, ), # 2
+    (3, TType.STRING, 'table', None, None, ), # 3
+    (4, TType.STRING, 'files', None, None, ), # 4
+    (5, TType.STRING, 'partition', None, None, ), # 5
+    (6, TType.STRING, 'tableObj', None, None, ), # 6
+    (7, TType.STRING, 'partitionObj', None, None, ), # 7
+  )
+
+  def __init__(self, writeId=None, database=None, table=None, files=None, partition=None, tableObj=None, partitionObj=None,):
+    self.writeId = writeId
+    self.database = database
+    self.table = table
+    self.files = files
+    self.partition = partition
+    self.tableObj = tableObj
+    self.partitionObj = partitionObj
+
+  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.I64:
+          self.writeId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.database = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.table = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.files = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.partition = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.tableObj = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.partitionObj = iprot.readString()
+        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('WriteEventInfo')
+    if self.writeId is not None:
+      oprot.writeFieldBegin('writeId', TType.I64, 1)
+      oprot.writeI64(self.writeId)
+      oprot.writeFieldEnd()
+    if self.database is not None:
+      oprot.writeFieldBegin('database', TType.STRING, 2)
+      oprot.writeString(self.database)
+      oprot.writeFieldEnd()
+    if self.table is not None:
+      oprot.writeFieldBegin('table', TType.STRING, 3)
+      oprot.writeString(self.table)
+      oprot.writeFieldEnd()
+    if self.files is not None:
+      oprot.writeFieldBegin('files', TType.STRING, 4)
+      oprot.writeString(self.files)
+      oprot.writeFieldEnd()
+    if self.partition is not None:
+      oprot.writeFieldBegin('partition', TType.STRING, 5)
+      oprot.writeString(self.partition)
+      oprot.writeFieldEnd()
+    if self.tableObj is not None:
+      oprot.writeFieldBegin('tableObj', TType.STRING, 6)
+      oprot.writeString(self.tableObj)
+      oprot.writeFieldEnd()
+    if self.partitionObj is not None:
+      oprot.writeFieldBegin('partitionObj', TType.STRING, 7)
+      oprot.writeString(self.partitionObj)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.writeId is None:
+      raise TProtocol.TProtocolException(message='Required field writeId is unset!')
+    if self.database is None:
+      raise TProtocol.TProtocolException(message='Required field database is unset!')
+    if self.table is None:
+      raise TProtocol.TProtocolException(message='Required field table is unset!')
+    if self.files is None:
+      raise TProtocol.TProtocolException(message='Required field files is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.writeId)
+    value = (value * 31) ^ hash(self.database)
+    value = (value * 31) ^ hash(self.table)
+    value = (value * 31) ^ hash(self.files)
+    value = (value * 31) ^ hash(self.partition)
+    value = (value * 31) ^ hash(self.tableObj)
+    value = (value * 31) ^ hash(self.partitionObj)
     return value
 
   def __repr__(self):
@@ -11703,10 +11876,10 @@ class ReplTblWriteIdStateRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype526, _size523) = iprot.readListBegin()
-          for _i527 in xrange(_size523):
-            _elem528 = iprot.readString()
-            self.partNames.append(_elem528)
+          (_etype533, _size530) = iprot.readListBegin()
+          for _i534 in xrange(_size530):
+            _elem535 = iprot.readString()
+            self.partNames.append(_elem535)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11743,8 +11916,8 @@ class ReplTblWriteIdStateRequest:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter529 in self.partNames:
-        oprot.writeString(iter529)
+      for iter536 in self.partNames:
+        oprot.writeString(iter536)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11814,10 +11987,10 @@ class GetValidWriteIdsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fullTableNames = []
-          (_etype533, _size530) = iprot.readListBegin()
-          for _i534 in xrange(_size530):
-            _elem535 = iprot.readString()
-            self.fullTableNames.append(_elem535)
+          (_etype540, _size537) = iprot.readListBegin()
+          for _i541 in xrange(_size537):
+            _elem542 = iprot.readString()
+            self.fullTableNames.append(_elem542)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11839,8 +12012,8 @@ class GetValidWriteIdsRequest:
     if self.fullTableNames is not None:
       oprot.writeFieldBegin('fullTableNames', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.fullTableNames))
-      for iter536 in self.fullTableNames:
-        oprot.writeString(iter536)
+      for iter543 in self.fullTableNames:
+        oprot.writeString(iter543)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -11923,10 +12096,10 @@ class TableValidWriteIds:
       elif fid == 3:
         if ftype == TType.LIST:
           self.invalidWriteIds = []
-          (_etype540, _size537) = iprot.readListBegin()
-          for _i541 in xrange(_size537):
-            _elem542 = iprot.readI64()
-            self.invalidWriteIds.append(_elem542)
+          (_etype547, _size544) = iprot.readListBegin()
+          for _i548 in xrange(_size544):
+            _elem549 = iprot.readI64()
+            self.invalidWriteIds.append(_elem549)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11961,8 +12134,8 @@ class TableValidWriteIds:
     if self.invalidWriteIds is not None:
       oprot.writeFieldBegin('invalidWriteIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.invalidWriteIds))
-      for iter543 in self.invalidWriteIds:
-        oprot.writeI64(iter543)
+      for iter550 in self.invalidWriteIds:
+        oprot.writeI64(iter550)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.minOpenWriteId is not None:
@@ -12034,11 +12207,11 @@ class GetValidWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.tblValidWriteIds = []
-          (_etype547, _size544) = iprot.readListBegin()
-          for _i548 in xrange(_size544):
-            _elem549 = TableValidWriteIds()
-            _elem549.read(iprot)
-            self.tblValidWriteIds.append(_elem549)
+          (_etype554, _size551) = iprot.readListBegin()
+          for _i555 in xrange(_size551):
+            _elem556 = TableValidWriteIds()
+            _elem556.read(iprot)
+            self.tblValidWriteIds.append(_elem556)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12055,8 +12228,8 @@ class GetValidWriteIdsResponse:
     if self.tblValidWriteIds is not None:
       oprot.writeFieldBegin('tblValidWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tblValidWriteIds))
-      for iter550 in self.tblValidWriteIds:
-        iter550.write(oprot)
+      for iter557 in self.tblValidWriteIds:
+        iter557.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12132,10 +12305,10 @@ class AllocateTableWriteIdsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.txnIds = []
-          (_etype554, _size551) = iprot.readListBegin()
-          for _i555 in xrange(_size551):
-            _elem556 = iprot.readI64()
-            self.txnIds.append(_elem556)
+          (_etype561, _size558) = iprot.readListBegin()
+          for _i562 in xrange(_size558):
+            _elem563 = iprot.readI64()
+            self.txnIds.append(_elem563)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12147,11 +12320,11 @@ class AllocateTableWriteIdsRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.srcTxnToWriteIdList = []
-          (_etype560, _size557) = iprot.readListBegin()
-          for _i561 in xrange(_size557):
-            _elem562 = TxnToWriteId()
-            _elem562.read(iprot)
-            self.srcTxnToWriteIdList.append(_elem562)
+          (_etype567, _size564) = iprot.readListBegin()
+          for _i568 in xrange(_size564):
+            _elem569 = TxnToWriteId()
+            _elem569.read(iprot)
+            self.srcTxnToWriteIdList.append(_elem569)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12176,8 +12349,8 @@ class AllocateTableWriteIdsRequest:
     if self.txnIds is not None:
       oprot.writeFieldBegin('txnIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.txnIds))
-      for iter563 in self.txnIds:
-        oprot.writeI64(iter563)
+      for iter570 in self.txnIds:
+        oprot.writeI64(iter570)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.replPolicy is not None:
@@ -12187,8 +12360,8 @@ class AllocateTableWriteIdsRequest:
     if self.srcTxnToWriteIdList is not None:
       oprot.writeFieldBegin('srcTxnToWriteIdList', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.srcTxnToWriteIdList))
-      for iter564 in self.srcTxnToWriteIdList:
-        iter564.write(oprot)
+      for iter571 in self.srcTxnToWriteIdList:
+        iter571.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12330,11 +12503,11 @@ class AllocateTableWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnToWriteIds = []
-          (_etype568, _size565) = iprot.readListBegin()
-          for _i569 in xrange(_size565):
-            _elem570 = TxnToWriteId()
-            _elem570.read(iprot)
-            self.txnToWriteIds.append(_elem570)
+          (_etype575, _size572) = iprot.readListBegin()
+          for _i576 in xrange(_size572):
+            _elem577 = TxnToWriteId()
+            _elem577.read(iprot)
+            self.txnToWriteIds.append(_elem577)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12351,8 +12524,8 @@ class AllocateTableWriteIdsResponse:
     if self.txnToWriteIds is not None:
       oprot.writeFieldBegin('txnToWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.txnToWriteIds))
-      for iter571 in self.txnToWriteIds:
-        iter571.write(oprot)
+      for iter578 in self.txnToWriteIds:
+        iter578.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12580,11 +12753,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype575, _size572) = iprot.readListBegin()
-          for _i576 in xrange(_size572):
-            _elem577 = LockComponent()
-            _elem577.read(iprot)
-            self.component.append(_elem577)
+          (_etype582, _size579) = iprot.readListBegin()
+          for _i583 in xrange(_size579):
+            _elem584 = LockComponent()
+            _elem584.read(iprot)
+            self.component.append(_elem584)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12621,8 +12794,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter578 in self.component:
-        iter578.write(oprot)
+      for iter585 in self.component:
+        iter585.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -13320,11 +13493,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype582, _size579) = iprot.readListBegin()
-          for _i583 in xrange(_size579):
-            _elem584 = ShowLocksResponseElement()
-            _elem584.read(iprot)
-            self.locks.append(_elem584)
+          (_etype589, _size586) = iprot.readListBegin()
+          for _i590 in xrange(_size586):
+            _elem591 = ShowLocksResponseElement()
+            _elem591.read(iprot)
+            self.locks.append(_elem591)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13341,8 +13514,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter585 in self.locks:
-        iter585.write(oprot)
+      for iter592 in self.locks:
+        iter592.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13557,20 +13730,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype589, _size586) = iprot.readSetBegin()
-          for _i590 in xrange(_size586):
-            _elem591 = iprot.readI64()
-            self.aborted.add(_elem591)
+          (_etype596, _size593) = iprot.readSetBegin()
+          for _i597 in xrange(_size593):
+            _elem598 = iprot.readI64()
+            self.aborted.add(_elem598)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype595, _size592) = iprot.readSetBegin()
-          for _i596 in xrange(_size592):
-            _elem597 = iprot.readI64()
-            self.nosuch.add(_elem597)
+          (_etype602, _size599) = iprot.readSetBegin()
+          for _i603 in xrange(_size599):
+            _elem604 = iprot.readI64()
+            self.nosuch.add(_elem604)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -13587,15 +13760,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter598 in self.aborted:
-        oprot.writeI64(iter598)
+      for iter605 in self.aborted:
+        oprot.writeI64(iter605)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter599 in self.nosuch:
-        oprot.writeI64(iter599)
+      for iter606 in self.nosuch:
+        oprot.writeI64(iter606)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13692,11 +13865,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype601, _vtype602, _size600 ) = iprot.readMapBegin()
-          for _i604 in xrange(_size600):
-            _key605 = iprot.readString()
-            _val606 = iprot.readString()
-            self.properties[_key605] = _val606
+          (_ktype608, _vtype609, _size607 ) = iprot.readMapBegin()
+          for _i611 in xrange(_size607):
+            _key612 = iprot.readString()
+            _val613 = iprot.readString()
+            self.properties[_key612] = _val613
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13733,9 +13906,9 @@ class CompactionRequest:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter607,viter608 in self.properties.items():
-        oprot.writeString(kiter607)
-        oprot.writeString(viter608)
+      for kiter614,viter615 in self.properties.items():
+        oprot.writeString(kiter614)
+        oprot.writeString(viter615)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14170,11 +14343,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype612, _size609) = iprot.readListBegin()
-          for _i613 in xrange(_size609):
-            _elem614 = ShowCompactResponseElement()
-            _elem614.read(iprot)
-            self.compacts.append(_elem614)
+          (_etype619, _size616) = iprot.readListBegin()
+          for _i620 in xrange(_size616):
+            _elem621 = ShowCompactResponseElement()
+            _elem621.read(iprot)
+            self.compacts.append(_elem621)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14191,8 +14364,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter615 in self.compacts:
-        iter615.write(oprot)
+      for iter622 in self.compacts:
+        iter622.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14281,10 +14454,10 @@ class AddDynamicPartitions:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype619, _size616) = iprot.readListBegin()
-          for _i620 in xrange(_size616):
-            _elem621 = iprot.readString()
-            self.partitionnames.append(_elem621)
+          (_etype626, _size623) = iprot.readListBegin()
+          for _i627 in xrange(_size623):
+            _elem628 = iprot.readString()
+            self.partitionnames.append(_elem628)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14322,8 +14495,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter622 in self.partitionnames:
-        oprot.writeString(iter622)
+      for iter629 in self.partitionnames:
+        oprot.writeString(iter629)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -14553,10 +14726,10 @@ class CreationMetadata:
       elif fid == 4:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype626, _size623) = iprot.readSetBegin()
-          for _i627 in xrange(_size623):
-            _elem628 = iprot.readString()
-            self.tablesUsed.add(_elem628)
+          (_etype633, _size630) = iprot.readSetBegin()
+          for _i634 in xrange(_size630):
+            _elem635 = iprot.readString()
+            self.tablesUsed.add(_elem635)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -14590,8 +14763,8 @@ class CreationMetadata:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 4)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter629 in self.tablesUsed:
-        oprot.writeString(iter629)
+      for iter636 in self.tablesUsed:
+        oprot.writeString(iter636)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -14903,11 +15076,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype633, _size630) = iprot.readListBegin()
-          for _i634 in xrange(_size630):
-            _elem635 = NotificationEvent()
-            _elem635.read(iprot)
-            self.events.append(_elem635)
+          (_etype640, _size637) = iprot.readListBegin()
+          for _i641 in xrange(_size637):
+            _elem642 = NotificationEvent()
+            _elem642.read(iprot)
+            self.events.append(_elem642)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14924,8 +15097,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter636 in self.events:
-        iter636.write(oprot)
+      for iter643 in self.events:
+        iter643.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15188,6 +15361,7 @@ class InsertEventRequestData:
    - replace
    - filesAdded
    - filesAddedChecksum
+   - subDirectoryList
   """
 
   thrift_spec = (
@@ -15195,12 +15369,14 @@ class InsertEventRequestData:
     (1, TType.BOOL, 'replace', None, None, ), # 1
     (2, TType.LIST, 'filesAdded', (TType.STRING,None), None, ), # 2
     (3, TType.LIST, 'filesAddedChecksum', (TType.STRING,None), None, ), # 3
+    (4, TType.LIST, 'subDirectoryList', (TType.STRING,None), None, ), # 4
   )
 
-  def __init__(self, replace=None, filesAdded=None, filesAddedChecksum=None,):
+  def __init__(self, replace=None, filesAdded=None, filesAddedChecksum=None, subDirectoryList=None,):
     self.replace = replace
     self.filesAdded = filesAdded
     self.filesAddedChecksum = filesAddedChecksum
+    self.subDirectoryList = subDirectoryList
 
   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:
@@ -15219,20 +15395,30 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype640, _size637) = iprot.readListBegin()
-          for _i641 in xrange(_size637):
-            _elem642 = iprot.readString()
-            self.filesAdded.append(_elem642)
+          (_etype647, _size644) = iprot.readListBegin()
+          for _i648 in xrange(_size644):
+            _elem649 = iprot.readString()
+            self.filesAdded.append(_elem649)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.filesAddedChecksum.append(_elem648)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readString()
+            self.filesAddedChecksum.append(_elem655)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.subDirectoryList = []
+          (_etype659, _size656) = iprot.readListBegin()
+          for _i660 in xrange(_size656):
+            _elem661 = iprot.readString()
+            self.subDirectoryList.append(_elem661)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15253,15 +15439,22 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter649 in self.filesAdded:
-        oprot.writeString(iter649)
+      for iter662 in self.filesAdded:
+        oprot.writeString(iter662)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter650 in self.filesAddedChecksum:
-        oprot.writeString(iter650)
+      for iter663 in self.filesAddedChecksum:
+        oprot.writeString(iter663)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.subDirectoryList is not None:
+      oprot.writeFieldBegin('subDirectoryList', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.subDirectoryList))
+      for iter664 in self.subDirectoryList:
+        oprot.writeString(iter664)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15278,6 +15471,7 @@ class InsertEventRequestData:
     value = (value * 31) ^ hash(self.replace)
     value = (value * 31) ^ hash(self.filesAdded)
     value = (value * 31) ^ hash(self.filesAddedChecksum)
+    value = (value * 31) ^ hash(self.subDirectoryList)
     return value
 
   def __repr__(self):
@@ -15419,10 +15613,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype654, _size651) = iprot.readListBegin()
-          for _i655 in xrange(_size651):
-            _elem656 = iprot.readString()
-            self.partitionVals.append(_elem656)
+          (_etype668, _size665) = iprot.readListBegin()
+          for _i669 in xrange(_size665):
+            _elem670 = iprot.readString()
+            self.partitionVals.append(_elem670)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15460,8 +15654,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter657 in self.partitionVals:
-        oprot.writeString(iter657)
+      for iter671 in self.partitionVals:
+        oprot.writeString(iter671)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.catName is not None:
@@ -15546,6 +15740,201 @@ class FireEventResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class WriteNotificationLogRequest:
+  """
+  Attributes:
+   - txnId
+   - writeId
+   - db
+   - table
+   - fileInfo
+   - partitionVals
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'txnId', None, None, ), # 1
+    (2, TType.I64, 'writeId', None, None, ), # 2
+    (3, TType.STRING, 'db', None, None, ), # 3
+    (4, TType.STRING, 'table', None, None, ), # 4
+    (5, TType.STRUCT, 'fileInfo', (InsertEventRequestData, InsertEventRequestData.thrift_spec), None, ), # 5
+    (6, TType.LIST, 'partitionVals', (TType.STRING,None), None, ), # 6
+  )
+
+  def __init__(self, txnId=None, writeId=None, db=None, table=None, fileInfo=None, partitionVals=None,):
+    self.txnId = txnId
+    self.writeId = writeId
+    self.db = db
+    self.table = table
+    self.fileInfo = fileInfo
+    self.partitionVals = partitionVals
+
+  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.I64:
+          self.txnId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.writeId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.table = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.fileInfo = InsertEventRequestData()
+          self.fileInfo.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.LIST:
+          self.partitionVals = []
+          (_etype675, _size672) = iprot.readListBegin()
+          for _i676 in xrange(_size672):
+            _elem677 = iprot.readString()
+            self.partitionVals.append(_elem677)
+          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('WriteNotificationLogRequest')
+    if self.txnId is not None:
+      oprot.writeFieldBegin('txnId', TType.I64, 1)
+      oprot.writeI64(self.txnId)
+      oprot.writeFieldEnd()
+    if self.writeId is not None:
+      oprot.writeFieldBegin('writeId', TType.I64, 2)
+      oprot.writeI64(self.writeId)
+      oprot.writeFieldEnd()
+    if self.db is not None:
+      oprot.writeFieldBegin('db', TType.STRING, 3)
+      oprot.writeString(self.db)
+      oprot.writeFieldEnd()
+    if self.table is not None:
+      oprot.writeFieldBegin('table', TType.STRING, 4)
+      oprot.writeString(self.table)
+      oprot.writeFieldEnd()
+    if self.fileInfo is not None:
+      oprot.writeFieldBegin('fileInfo', TType.STRUCT, 5)
+      self.fileInfo.write(oprot)
+      oprot.writeFieldEnd()
+    if self.partitionVals is not None:
+      oprot.writeFieldBegin('partitionVals', TType.LIST, 6)
+      oprot.writeListBegin(TType.STRING, len(self.partitionVals))
+      for iter678 in self.partitionVals:
+        oprot.writeString(iter678)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.txnId is None:
+      raise TProtocol.TProtocolException(message='Required field txnId is unset!')
+    if self.writeId is None:
+      raise TProtocol.TProtocolException(message='Required field writeId is unset!')
+    if self.db is None:
+      raise TProtocol.TProtocolException(message='Required field db is unset!')
+    if self.table is None:
+      raise TProtocol.TProtocolException(message='Required field table is unset!')
+    if self.fileInfo is None:
+      raise TProtocol.TProtocolException(message='Required field fileInfo is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.txnId)
+    value = (value * 31) ^ hash(self.writeId)
+    value = (value * 31) ^ hash(self.db)
+    value = (value * 31) ^ hash(self.table)
+    value = (value * 31) ^ hash(self.fileInfo)
+    value = (value * 31) ^ hash(self.partitionVals)
+    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 WriteNotificationLogResponse:
+
+  thrift_spec = (
+  )
+
+  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
+      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('WriteNotificationLogResponse')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    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 MetadataPpdResult:
   """
   Attributes:
@@ -15653,12 +16042,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype659, _vtype660, _size658 ) = iprot.readMapBegin()
-          for _i662 in xrange(_size658):
-            _key663 = iprot.readI64()
-            _val664 = MetadataPpdResult()
-            _val664.read(iprot)
-            self.metadata[_key663] = _val664
+          (_ktype680, _vtype681, _size679 ) = iprot.readMapBegin()
+          for _i683 in xrange(_size679):
+            _key684 = iprot.readI64()
+            _val685 = MetadataPpdResult()
+            _val685.read(iprot)
+            self.metadata[_key684] = _val685
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15680,9 +16069,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter665,viter666 in self.metadata.items():
-        oprot.writeI64(kiter665)
-        viter666.write(oprot)
+      for kiter686,viter687 in self.metadata.items():
+        oprot.writeI64(kiter686)
+        viter687.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15752,10 +16141,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype670, _size667) = iprot.readListBegin()
-          for _i671 in xrange(_size667):
-            _elem672 = iprot.readI64()
-            self.fileIds.append(_elem672)
+          (_etype691, _size688) = iprot.readListBegin()
+          for _i692 in xrange(_size688):
+            _elem693 = iprot.readI64()
+            self.fileIds.append(_elem693)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15787,8 +16176,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter673 in self.fileIds:
-        oprot.writeI64(iter673)
+      for iter694 in self.fileIds:
+        oprot.writeI64(iter694)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -15862,11 +16251,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype675, _vtype676, _size674 ) = iprot.readMapBegin()
-          for _i678 in xrange(_size674):
-            _key679 = iprot.readI64()
-            _val680 = iprot.readString()
-            self.metadata[_key679] = _val680
+          (_ktype696, _vtype697, _size695 ) = iprot.readMapBegin()
+          for _i699 in xrange(_size695):
+            _key700 = iprot.readI64()
+            _val701 = iprot.readString()
+            self.metadata[_key700] = _val701
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15888,9 +16277,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter681,viter682 in self.metadata.items():
-        oprot.writeI64(kiter681)
-        oprot.writeString(viter682)
+      for kiter702,viter703 in self.metadata.items():
+        oprot.writeI64(kiter702)
+        oprot.writeString(viter703)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15951,10 +16340,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype686, _size683) = iprot.readListBegin()
-          for _i687 in xrange(_size683):
-            _elem688 = iprot.readI64()
-            self.fileIds.append(_elem688)
+          (_etype707, _size704) = iprot.readListBegin()
+          for _i708 in xrange(_size704):
+            _elem709 = iprot.readI64()
+            self.fileIds.append(_elem709)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15971,8 +16360,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter689 in self.fileIds:
-        oprot.writeI64(iter689)
+      for iter710 in self.fileIds:
+        oprot.writeI64(iter710)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16078,20 +16467,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype693, _size690) = iprot.readListBegin()
-          for _i694 in xrange(_size690):
-            _elem695 = iprot.readI64()
-            self.fileIds.append(_elem695)
+          (_etype714, _size711) = iprot.readListBegin()
+          for _i715 in xrange(_size711):
+            _elem716 = iprot.readI64()
+            self.fileIds.append(_elem716)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype699, _size696) = iprot.readListBegin()
-          for _i700 in xrange(_size696):
-            _elem701 = iprot.readString()
-            self.metadata.append(_elem701)
+          (_etype720, _size717) = iprot.readListBegin()
+          for _i721 in xrange(_size717):
+            _elem722 = iprot.readString()
+            self.metadata.append(_elem722)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16113,15 +16502,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter702 in self.fileIds:
-        oprot.writeI64(iter702)
+      for iter723 in self.fileIds:
+        oprot.writeI64(iter723)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter703 in self.metadata:
-        oprot.writeString(iter703)
+      for iter724 in self.metadata:
+        oprot.writeString(iter724)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -16229,10 +16618,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype707, _size704) = iprot.readListBegin()
-          for _i708 in xrange(_size704):
-            _elem709 = iprot.readI64()
-            self.fileIds.append(_elem709)
+          (_etype728, _size725) = iprot.readListBegin()
+          for _i729 in xrange(_size725):
+            _elem730 = iprot.readI64()
+            self.fileIds.append(_elem730)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16249,8 +16638,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter710 in self.fileIds:
-        oprot.writeI64(iter710)
+      for iter731 in self.fileIds:
+        oprot.writeI64(iter731)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16479,11 +16868,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype714, _size711) = iprot.readListBegin()
-          for _i715 in xrange(_size711):
-            _elem716 = Function()
-            _elem716.read(iprot)
-            self.functions.append(_elem716)
+          (_etype735, _size732) = iprot.readListBegin()
+          for _i736 in xrange(_size732):
+            _elem737 = Function()
+            _elem737.read(iprot)
+            self.functions.append(_elem737)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16500,8 +16889,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter717 in self.functions:
-        iter717.write(oprot)
+      for iter738 in self.functions:
+        iter738.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16553,10 +16942,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype721, _size718) = iprot.readListBegin()
-          for _i722 in xrange(_size718):
-            _elem723 = iprot.readI32()
-            self.values.append(_elem723)
+          (_etype742, _size739) = iprot.readListBegin()
+          for _i743 in xrange(_size739):
+            _elem744 = iprot.readI32()
+            self.values.append(_elem744)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16573,8 +16962,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter724 in self.values:
-        oprot.writeI32(iter724)
+      for iter745 in self.values:
+        oprot.writeI32(iter745)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16819,10 +17208,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype728, _size725) = iprot.readListBegin()
-          for _i729 in xrange(_size725):
-            _elem730 = iprot.readString()
-            self.tblNames.append(_elem730)
+          (_etype749, _size746) = iprot.readListBegin()
+          for _i750 in xrange(_size746):
+            _elem751 = iprot.readString()
+            self.tblNames.append(_elem751)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16854,8 +17243,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter731 in self.tblNames:
-        oprot.writeString(iter731)
+      for iter752 in self.tblNames:
+        oprot.writeString(iter752)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -16920,11 +17309,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype735, _size732) = iprot.readListBegin()
-          for _i736 in xrange(_size732):
-            _elem737 = Table()
-            _elem737.read(iprot)
-            self.tables.append(_elem737)
+          (_etype756, _size753) = iprot.readListBegin()
+          for _i757 in xrange(_size753):
+            _elem758 = Table()
+            _elem758.read(iprot)
+            self.tables.append(_elem758)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16941,8 +17330,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter738 in self.tables:
-        iter738.write(oprot)
+      for iter759 in self.tables:
+        iter759.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17256,10 +17645,10 @@ class Materialization:
       if fid == 1:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype742, _size739) = iprot.readSetBegin()
-          for _i743 in xrange(_size739):
-            _elem744 = iprot.readString()
-            self.tablesUsed.add(_elem744)
+          (_etype763, _size760) = iprot.readSetBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = iprot.readString()
+            self.tablesUsed.add(_elem765)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -17291,8 +17680,8 @@ class Materialization:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 1)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter745 in self.tablesUsed:
-        oprot.writeString(iter745)
+      for iter766 in self.tablesUsed:
+        oprot.writeString(iter766)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -18197,44 +18586,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype749, _size746) = iprot.readListBegin()
-          for _i750 in xrange(_size746):
-            _elem751 = WMPool()
-            _elem751.read(iprot)
-            self.pools.append(_elem751)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = WMPool()
+            _elem772.read(iprot)
+            self.pools.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = WMMapping()
-            _elem757.read(iprot)
-            self.mappings.append(_elem757)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = WMMapping()
+            _elem778.read(iprot)
+            self.mappings.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype761, _size758) = iprot.readListBegin()
-          for _i762 in xrange(_size758):
-            _elem763 = WMTrigger()
-            _elem763.read(iprot)
-            self.triggers.append(_elem763)
+          (_etype782, _size779) = iprot.readListBegin()
+          for _i783 in xrange(_size779):
+            _elem784 = WMTrigger()
+            _elem784.read(iprot)
+            self.triggers.append(_elem784)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype767, _size764) = iprot.readListBegin()
-          for _i768 in xrange(_size764):
-            _elem769 = WMPoolTrigger()
-            _elem769.read(iprot)
-            self.poolTriggers.append(_elem769)
+          (_etype788, _size785) = iprot.readListBegin()
+          for _i789 in xrange(_size785):
+            _elem790 = WMPoolTrigger()
+            _elem790.read(iprot)
+            self.poolTriggers.append(_elem790)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18255,29 +18644,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter770 in self.pools:
-        iter770.write(oprot)
+      for iter791 in self.pools:
+        iter791.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter771 in self.mappings:
-        iter771.write(oprot)
+      for iter792 in self.mappings:
+        iter792.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter772 in self.triggers:
-        iter772.write(oprot)
+      for iter793 in self.triggers:
+        iter793.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter773 in self.poolTriggers:
-        iter773.write(oprot)
+      for iter794 in self.poolTriggers:
+        iter794.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18751,11 +19140,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype777, _size774) = iprot.readListBegin()
-          for _i778 in xrange(_size774):
-            _elem779 = WMResourcePlan()
-            _elem779.read(iprot)
-            self.resourcePlans.append(_elem779)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = WMResourcePlan()
+            _elem800.read(iprot)
+            self.resourcePlans.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18772,8 +19161,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter780 in self.resourcePlans:
-        iter780.write(oprot)
+      for iter801 in self.resourcePlans:
+        iter801.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19077,20 +19466,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype784, _size781) = iprot.readListBegin()
-          for _i785 in xrange(_size781):
-            _elem786 = iprot.readString()
-            self.errors.append(_elem786)
+          (_etype805, _size802) = iprot.readListBegin()
+          for _i806 in xrange(_size802):
+            _elem807 = iprot.readString()
+            self.errors.append(_elem807)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = iprot.readString()
-            self.warnings.append(_elem792)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.warnings.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19107,15 +19496,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter793 in self.errors:
-        oprot.writeString(iter793)
+      for iter814 in self.errors:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter794 in self.warnings:
-        oprot.writeString(iter794)
+      for iter815 in self.warnings:
+        oprot.writeString(iter815)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19692,11 +20081,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype798, _size795) = iprot.readListBegin()
-          for _i799 in xrange(_size795):
-            _elem800 = WMTrigger()
-            _elem800.read(iprot)
-            self.triggers.append(_elem800)
+          (_etype819, _size816) = iprot.readListBegin()
+          for _i820 in xrange(_size816):
+            _elem821 = WMTrigger()
+            _elem821.read(iprot)
+            self.triggers.append(_elem821)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19713,8 +20102,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter801 in self.triggers:
-        iter801.write(oprot)
+      for iter822 in self.triggers:
+        iter822.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20898,11 +21287,11 @@ class SchemaVersion:
       elif fid == 4:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype805, _size802) = iprot.readListBegin()
-          for _i806 in xrange(_size802):
-            _elem807 = FieldSchema()
-            _elem807.read(iprot)
-            self.cols.append(_elem807)
+          (_etype826, _size823) = iprot.readListBegin()
+          for _i827 in xrange(_size823):
+            _elem828 = FieldSchema()
+            _elem828.read(iprot)
+            self.cols.append(_elem828)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20962,8 +21351,8 @@ class SchemaVersion:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter808 in self.cols:
-        iter808.write(oprot)
+      for iter829 in self.cols:
+        iter829.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.state is not None:
@@ -21218,11 +21607,11 @@ class FindSchemasByColsResp:
       if fid == 1:
         if ftype == TType.LIST:
           self.schemaVersions = []
-          (_etype812, _size809) = iprot.readListBegin()
-          for _i813 in xrange(_size809):
-            _elem814 = SchemaVersionDescriptor()
-            _elem814.read(iprot)
-            self.schemaVersions.append(_elem814)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = SchemaVersionDescriptor()
+            _elem835.read(iprot)
+            self.schemaVersions.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21239,8 +21628,8 @@ class FindSchemasByColsResp:
     if self.schemaVersions is not None:
       oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
-      for iter815 in self.schemaVersions:
-        iter815.write(oprot)
+      for iter836 in self.schemaVersions:
+        iter836.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index fc640d0..0348ff2 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2582,10 +2582,12 @@ class CommitTxnRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   TXNID = 1
   REPLPOLICY = 2
+  WRITEEVENTINFOS = 3
 
   FIELDS = {
     TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid'},
-    REPLPOLICY => {:type => ::Thrift::Types::STRING, :name => 'replPolicy', :optional => true}
+    REPLPOLICY => {:type => ::Thrift::Types::STRING, :name => 'replPolicy', :optional => true},
+    WRITEEVENTINFOS => {:type => ::Thrift::Types::LIST, :name => 'writeEventInfos', :element => {:type => ::Thrift::Types::STRUCT, :class => ::WriteEventInfo}, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2597,6 +2599,38 @@ class CommitTxnRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class WriteEventInfo
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  WRITEID = 1
+  DATABASE = 2
+  TABLE = 3
+  FILES = 4
+  PARTITION = 5
+  TABLEOBJ = 6
+  PARTITIONOBJ = 7
+
+  FIELDS = {
+    WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId'},
+    DATABASE => {:type => ::Thrift::Types::STRING, :name => 'database'},
+    TABLE => {:type => ::Thrift::Types::STRING, :name => 'table'},
+    FILES => {:type => ::Thrift::Types::STRING, :name => 'files'},
+    PARTITION => {:type => ::Thrift::Types::STRING, :name => 'partition', :optional => true},
+    TABLEOBJ => {:type => ::Thrift::Types::STRING, :name => 'tableObj', :optional => true},
+    PARTITIONOBJ => {:type => ::Thrift::Types::STRING, :name => 'partitionObj', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field writeId is unset!') unless @writeId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field database is unset!') unless @database
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field table is unset!') unless @table
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field files is unset!') unless @files
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class ReplTblWriteIdStateRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   VALIDWRITEIDLIST = 1
@@ -3395,11 +3429,13 @@ class InsertEventRequestData
   REPLACE = 1
   FILESADDED = 2
   FILESADDEDCHECKSUM = 3
+  SUBDIRECTORYLIST = 4
 
   FIELDS = {
     REPLACE => {:type => ::Thrift::Types::BOOL, :name => 'replace', :optional => true},
     FILESADDED => {:type => ::Thrift::Types::LIST, :name => 'filesAdded', :element => {:type => ::Thrift::Types::STRING}},
-    FILESADDEDCHECKSUM => {:type => ::Thrift::Types::LIST, :name => 'filesAddedChecksum', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+    FILESADDEDCHECKSUM => {:type => ::Thrift::Types::LIST, :name => 'filesAddedChecksum', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    SUBDIRECTORYLIST => {:type => ::Thrift::Types::LIST, :name => 'subDirectoryList', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3477,6 +3513,52 @@ class FireEventResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class WriteNotificationLogRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TXNID = 1
+  WRITEID = 2
+  DB = 3
+  TABLE = 4
+  FILEINFO = 5
+  PARTITIONVALS = 6
+
+  FIELDS = {
+    TXNID => {:type => ::Thrift::Types::I64, :name => 'txnId'},
+    WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId'},
+    DB => {:type => ::Thrift::Types::STRING, :name => 'db'},
+    TABLE => {:type => ::Thrift::Types::STRING, :name => 'table'},
+    FILEINFO => {:type => ::Thrift::Types::STRUCT, :name => 'fileInfo', :class => ::InsertEventRequestData},
+    PARTITIONVALS => {:type => ::Thrift::Types::LIST, :name => 'partitionVals', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txnId is unset!') unless @txnId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field writeId is unset!') unless @writeId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db is unset!') unless @db
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field table is unset!') unless @table
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileInfo is unset!') unless @fileInfo
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class WriteNotificationLogResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+
+  FIELDS = {
+
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class MetadataPpdResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   METADATA = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index bbf3f12..2bd958e 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2751,6 +2751,21 @@ module ThriftHiveMetastore
       return
     end
 
+    def add_write_notification_log(rqst)
+      send_add_write_notification_log(rqst)
+      return recv_add_write_notification_log()
+    end
+
+    def send_add_write_notification_log(rqst)
+      send_message('add_write_notification_log', Add_write_notification_log_args, :rqst => rqst)
+    end
+
+    def recv_add_write_notification_log()
+      result = receive_message(Add_write_notification_log_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'add_write_notification_log failed: unknown result')
+    end
+
     def cm_recycle(request)
       send_cm_recycle(request)
       return recv_cm_recycle()
@@ -5520,6 +5535,13 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'flushCache', seqid)
     end
 
+    def process_add_write_notification_log(seqid, iprot, oprot)
+      args = read_args(iprot, Add_write_notification_log_args)
+      result = Add_write_notification_log_result.new()
+      result.success = @handler.add_write_notification_log(args.rqst)
+      write_result(result, oprot, 'add_write_notification_log', seqid)
+    end
+
     def process_cm_recycle(seqid, iprot, oprot)
       args = read_args(iprot, Cm_recycle_args)
       result = Cm_recycle_result.new()
@@ -12220,6 +12242,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Add_write_notification_log_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = -1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::WriteNotificationLogRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_write_notification_log_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::WriteNotificationLogResponse}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Cm_recycle_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     REQUEST = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 1327fa2..c6c04b7 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -81,6 +81,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.StatsUpdateMode;
@@ -7169,6 +7170,55 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public void commit_txn(CommitTxnRequest rqst) throws TException {
+      // in replication flow, the write notification log table will be updated here.
+      if (rqst.isSetWriteEventInfos()) {
+        long targetTxnId = getTxnHandler().getTargetTxnId(rqst.getReplPolicy(), rqst.getTxnid());
+        if (targetTxnId < 0) {
+          //looks like a retry
+          return;
+        }
+        for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
+          String[] filesAdded = ReplChangeManager.getListFromSeparatedString(writeEventInfo.getFiles());
+          List<String> partitionValue = null;
+          Partition ptnObj = null;
+          String root;
+          Table tbl = getTblObject(writeEventInfo.getDatabase(), writeEventInfo.getTable());
+
+          if (writeEventInfo.getPartition() != null && !writeEventInfo.getPartition().isEmpty()) {
+            partitionValue = Warehouse.getPartValuesFromPartName(writeEventInfo.getPartition());
+            ptnObj = getPartitionObj(writeEventInfo.getDatabase(), writeEventInfo.getTable(), partitionValue, tbl);
+            root = ptnObj.getSd().getLocation();
+          } else {
+            root = tbl.getSd().getLocation();
+          }
+
+          InsertEventRequestData insertData = new InsertEventRequestData();
+          insertData.setReplace(true);
+
+          // The files in the commit txn message during load will have files with path corresponding to source
+          // warehouse. Need to transform them to target warehouse using table or partition object location.
+          for (String file : filesAdded) {
+            String[] decodedPath = ReplChangeManager.decodeFileUri(file);
+            String name = (new Path(decodedPath[0])).getName();
+            Path newPath = FileUtils.getTransformedPath(name, decodedPath[3], root);
+            insertData.addToFilesAdded(newPath.toUri().toString());
+            insertData.addToSubDirectoryList(decodedPath[3]);
+            try {
+              insertData.addToFilesAddedChecksum(ReplChangeManager.checksumFor(newPath, newPath.getFileSystem(conf)));
+            } catch (IOException e) {
+              LOG.error("failed to get checksum for the file " + newPath + " with error: " + e.getMessage());
+              throw new TException(e.getMessage());
+            }
+          }
+
+          WriteNotificationLogRequest wnRqst = new WriteNotificationLogRequest(targetTxnId,
+                  writeEventInfo.getWriteId(), writeEventInfo.getDatabase(), writeEventInfo.getTable(), insertData);
+          if (partitionValue != null) {
+            wnRqst.setPartitionVals(partitionValue);
+          }
+          addTxnWriteNotificationLog(tbl, ptnObj, wnRqst);
+        }
+      }
       getTxnHandler().commitTxn(rqst);
       if (listeners != null && !listeners.isEmpty()) {
         MetaStoreListenerNotifier.notifyEvent(listeners, EventType.COMMIT_TXN,
@@ -7198,6 +7248,42 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return response;
     }
 
+    private void addTxnWriteNotificationLog(Table tableObj, Partition ptnObj, WriteNotificationLogRequest rqst)
+            throws MetaException {
+      String partition = ""; //Empty string is an invalid partition name. Can be used for non partitioned table.
+      if (ptnObj != null) {
+        partition = Warehouse.makePartName(tableObj.getPartitionKeys(), rqst.getPartitionVals());
+      }
+      AcidWriteEvent event = new AcidWriteEvent(partition, tableObj, ptnObj, rqst);
+      getTxnHandler().addWriteNotificationLog(event);
+      if (listeners != null && !listeners.isEmpty()) {
+        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ACID_WRITE, event);
+      }
+    }
+
+    private Table getTblObject(String db, String table) throws MetaException, NoSuchObjectException {
+      GetTableRequest req = new GetTableRequest(db, table);
+      req.setCapabilities(new ClientCapabilities(Lists.newArrayList(ClientCapability.TEST_CAPABILITY)));
+      return get_table_req(req).getTable();
+    }
+
+    private Partition getPartitionObj(String db, String table, List<String> partitionVals, Table tableObj)
+            throws MetaException, NoSuchObjectException {
+      if (tableObj.isSetPartitionKeys() && !tableObj.getPartitionKeys().isEmpty()) {
+        return get_partition(db, table, partitionVals);
+      }
+      return null;
+    }
+
+    @Override
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)
+            throws MetaException, NoSuchObjectException {
+      Table tableObj = getTblObject(rqst.getDb(), rqst.getTable());
+      Partition ptnObj = getPartitionObj(rqst.getDb(), rqst.getTable(), rqst.getPartitionVals(), tableObj);
+      addTxnWriteNotificationLog(tableObj, ptnObj, rqst);
+      return new WriteNotificationLogResponse();
+    }
+
     @Override
     public LockResponse lock(LockRequest rqst) throws TException {
       return getTxnHandler().lock(rqst);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index da41e6e..bfd7141 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2506,10 +2506,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public void replCommitTxn(long srcTxnId, String replPolicy)
+  public void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException {
-    CommitTxnRequest rqst = new CommitTxnRequest(srcTxnId);
-    rqst.setReplPolicy(replPolicy);
     client.commit_txn(rqst);
   }
 
@@ -2756,6 +2754,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return client.fire_listener_event(rqst);
   }
 
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  @Override
+  public void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.add_write_notification_log(rqst);
+  }
+
   /**
    * Creates a synchronized wrapper for any {@link IMetaStoreClient}.
    * This may be used by multi-threaded applications until we have

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index bc09076..b5d147b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
 import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
@@ -125,6 +126,7 @@ import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
@@ -2871,8 +2873,8 @@ public interface IMetaStoreClient {
   /**
    * Commit a transaction.  This will also unlock any locks associated with
    * this transaction.
-   * @param srcTxnid id of transaction at source which is committed and to be replicated.
-   * @param replPolicy the replication policy to identify the source cluster
+   * @param rqst Information containing the txn info and write event information
+   * of transaction at source which is committed and to be replicated
    * @throws NoSuchTxnException if the requested transaction does not exist.
    * This can result fro the transaction having timed out and been deleted by
    * the compactor.
@@ -2880,7 +2882,7 @@ public interface IMetaStoreClient {
    * aborted.  This can result from the transaction timing out.
    * @throws TException
    */
-  void replCommitTxn(long srcTxnid, String replPolicy)
+  void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException;
 
   /**
@@ -3193,6 +3195,14 @@ public interface IMetaStoreClient {
   @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
   FireEventResponse fireListenerEvent(FireEventRequest request) throws TException;
 
+  /**
+   * Add a event related to write operations in an ACID table.
+   * @param rqst message containing information for acid write operation.
+   * @throws TException
+   */
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException;
+
   class IncompatibleMetastoreException extends MetaException {
     IncompatibleMetastoreException(String message) {
       super(message);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index e0e65cf..de226bf 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import java.sql.Connection;
 
@@ -282,6 +283,17 @@ public abstract class MetaStoreEventListener implements Configurable {
           throws MetaException {
   }
 
+  /**
+   * This will be called to perform acid write operation.
+   * @param acidWriteEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
+   * @throws MetaException
+   */
+  public void onAcidWrite(AcidWriteEvent acidWriteEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+  }
+
   @Override
   public Configuration getConf() {
     return this.conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index 3cf8314..c296f57 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import java.sql.Connection;
 import java.util.List;
@@ -221,6 +222,8 @@ public class MetaStoreListenerNotifier {
               (listener, event) -> listener.onAbortTxn((AbortTxnEvent) event, null, null))
           .put(EventType.ALLOC_WRITE_ID,
               (listener, event) -> listener.onAllocWriteId((AllocWriteIdEvent) event, null, null))
+          .put(EventType.ACID_WRITE,
+                  (listener, event) -> listener.onAcidWrite((AcidWriteEvent) event, null, null))
           .build()
   );
 
@@ -241,6 +244,9 @@ public class MetaStoreListenerNotifier {
       .put(EventType.ALLOC_WRITE_ID,
         (listener, event, dbConn, sqlGenerator) ->
                 listener.onAllocWriteId((AllocWriteIdEvent) event, dbConn, sqlGenerator))
+      .put(EventType.ACID_WRITE,
+        (listener, event, dbConn, sqlGenerator) ->
+                listener.onAcidWrite((AcidWriteEvent) event, dbConn, sqlGenerator))
       .build()
   );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 7490243..8721022 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -159,6 +159,7 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
@@ -206,6 +207,7 @@ import org.apache.hadoop.hive.metastore.model.MWMPool;
 import org.apache.hadoop.hive.metastore.model.MWMResourcePlan;
 import org.apache.hadoop.hive.metastore.model.MWMResourcePlan.Status;
 import org.apache.hadoop.hive.metastore.model.MWMTrigger;
+import org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -9611,6 +9613,64 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      long tmp = System.currentTimeMillis() / 1000 - olderThan;
+      int tooOld = (tmp > Integer.MAX_VALUE) ? 0 : (int) tmp;
+      query = pm.newQuery(MTxnWriteNotificationLog.class, "eventTime < tooOld");
+      query.declareParameters("java.lang.Integer tooOld");
+      Collection<MTxnWriteNotificationLog> toBeRemoved = (Collection) query.execute(tooOld);
+      if (CollectionUtils.isNotEmpty(toBeRemoved)) {
+        pm.deletePersistentAll(toBeRemoved);
+      }
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    List<WriteEventInfo> writeEventInfoList = null;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      List<String> parameterVals = new ArrayList<>();
+      StringBuilder filterBuilder = new StringBuilder(" txnId == " + Long.toString(txnId));
+      if (dbName != null && !"*".equals(dbName)) { // * means get all database, so no need to add filter
+        appendSimpleCondition(filterBuilder, "database", new String[]{dbName}, parameterVals);
+      }
+      if (tableName != null && !"*".equals(tableName)) {
+        appendSimpleCondition(filterBuilder, "table", new String[]{tableName}, parameterVals);
+      }
+      query = pm.newQuery(MTxnWriteNotificationLog.class, filterBuilder.toString());
+      query.setOrdering("database,table ascending");
+      List<MTxnWriteNotificationLog> mplans = (List<MTxnWriteNotificationLog>)query.executeWithArray(
+              parameterVals.toArray(new String[parameterVals.size()]));
+      pm.retrieveAll(mplans);
+      commited = commitTransaction();
+      if (mplans != null && mplans.size() > 0) {
+        writeEventInfoList = Lists.newArrayList();
+        for (MTxnWriteNotificationLog mplan : mplans) {
+          WriteEventInfo writeEventInfo = new WriteEventInfo(mplan.getWriteId(), mplan.getDatabase(),
+                  mplan.getTable(), mplan.getFiles());
+          writeEventInfo.setPartition(mplan.getPartition());
+          writeEventInfo.setPartitionObj(mplan.getPartObject());
+          writeEventInfo.setTableObj(mplan.getTableObject());
+          writeEventInfoList.add(writeEventInfo);
+        }
+      }
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return writeEventInfoList;
+  }
+
   private void prepareQuotes() throws SQLException {
     if (dbType == DatabaseProduct.MYSQL) {
       assert pm.currentTransaction().isActive();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index d019941..73a518d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
@@ -1665,4 +1666,17 @@ public interface RawStore extends Configurable {
   Map<String, List<String>> getPartitionColsWithStats(String catName, String dbName,
       String tableName) throws MetaException, NoSuchObjectException;
 
+  /**
+   * Remove older notification events.
+   * @param olderThan Remove any events older than a given number of seconds
+   */
+  void cleanWriteNotificationEvents(int olderThan);
+
+  /**
+   * Get all write events for a specific transaction .
+   * @param txnId get all the events done by this transaction
+   * @param dbName the name of db for which dump is being taken
+   * @param tableName the name of the table for which the dump is being taken
+   */
+  List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException;
 }


[06/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index ec26cca..a83017b 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1271,6 +1271,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function flushCache();
   /**
+   * @param \metastore\WriteNotificationLogRequest $rqst
+   * @return \metastore\WriteNotificationLogResponse
+   */
+  public function add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst);
+  /**
    * @param \metastore\CmRecycleRequest $request
    * @return \metastore\CmRecycleResponse
    * @throws \metastore\MetaException
@@ -10933,6 +10938,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst)
+  {
+    $this->send_add_write_notification_log($rqst);
+    return $this->recv_add_write_notification_log();
+  }
+
+  public function send_add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_write_notification_log_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_, 'add_write_notification_log', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_write_notification_log', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_write_notification_log()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_write_notification_log_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_add_write_notification_log_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("add_write_notification_log failed: unknown result");
+  }
+
   public function cm_recycle(\metastore\CmRecycleRequest $request)
   {
     $this->send_cm_recycle($request);
@@ -15440,14 +15496,14 @@ class ThriftHiveMetastore_get_databases_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)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem825 = null;
-              $xfer += $input->readString($elem825);
-              $this->success []= $elem825;
+              $elem846 = null;
+              $xfer += $input->readString($elem846);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15483,9 +15539,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter826)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $output->writeString($iter826);
+            $xfer += $output->writeString($iter847);
           }
         }
         $output->writeListEnd();
@@ -15616,14 +15672,14 @@ class ThriftHiveMetastore_get_all_databases_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)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem832 = null;
-              $xfer += $input->readString($elem832);
-              $this->success []= $elem832;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->success []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15659,9 +15715,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter833)
+          foreach ($this->success as $iter854)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -16662,18 +16718,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size834 = 0;
-            $_ktype835 = 0;
-            $_vtype836 = 0;
-            $xfer += $input->readMapBegin($_ktype835, $_vtype836, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size855 = 0;
+            $_ktype856 = 0;
+            $_vtype857 = 0;
+            $xfer += $input->readMapBegin($_ktype856, $_vtype857, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $key839 = '';
-              $val840 = new \metastore\Type();
-              $xfer += $input->readString($key839);
-              $val840 = new \metastore\Type();
-              $xfer += $val840->read($input);
-              $this->success[$key839] = $val840;
+              $key860 = '';
+              $val861 = new \metastore\Type();
+              $xfer += $input->readString($key860);
+              $val861 = new \metastore\Type();
+              $xfer += $val861->read($input);
+              $this->success[$key860] = $val861;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16709,10 +16765,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter841 => $viter842)
+          foreach ($this->success as $kiter862 => $viter863)
           {
-            $xfer += $output->writeString($kiter841);
-            $xfer += $viter842->write($output);
+            $xfer += $output->writeString($kiter862);
+            $xfer += $viter863->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16916,15 +16972,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size843 = 0;
-            $_etype846 = 0;
-            $xfer += $input->readListBegin($_etype846, $_size843);
-            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
+            $_size864 = 0;
+            $_etype867 = 0;
+            $xfer += $input->readListBegin($_etype867, $_size864);
+            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
             {
-              $elem848 = null;
-              $elem848 = new \metastore\FieldSchema();
-              $xfer += $elem848->read($input);
-              $this->success []= $elem848;
+              $elem869 = null;
+              $elem869 = new \metastore\FieldSchema();
+              $xfer += $elem869->read($input);
+              $this->success []= $elem869;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16976,9 +17032,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter849)
+          foreach ($this->success as $iter870)
           {
-            $xfer += $iter849->write($output);
+            $xfer += $iter870->write($output);
           }
         }
         $output->writeListEnd();
@@ -17220,15 +17276,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size850 = 0;
-            $_etype853 = 0;
-            $xfer += $input->readListBegin($_etype853, $_size850);
-            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
+            $_size871 = 0;
+            $_etype874 = 0;
+            $xfer += $input->readListBegin($_etype874, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $elem855 = null;
-              $elem855 = new \metastore\FieldSchema();
-              $xfer += $elem855->read($input);
-              $this->success []= $elem855;
+              $elem876 = null;
+              $elem876 = new \metastore\FieldSchema();
+              $xfer += $elem876->read($input);
+              $this->success []= $elem876;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17280,9 +17336,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter856)
+          foreach ($this->success as $iter877)
           {
-            $xfer += $iter856->write($output);
+            $xfer += $iter877->write($output);
           }
         }
         $output->writeListEnd();
@@ -17496,15 +17552,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size857 = 0;
-            $_etype860 = 0;
-            $xfer += $input->readListBegin($_etype860, $_size857);
-            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
+            $_size878 = 0;
+            $_etype881 = 0;
+            $xfer += $input->readListBegin($_etype881, $_size878);
+            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
             {
-              $elem862 = null;
-              $elem862 = new \metastore\FieldSchema();
-              $xfer += $elem862->read($input);
-              $this->success []= $elem862;
+              $elem883 = null;
+              $elem883 = new \metastore\FieldSchema();
+              $xfer += $elem883->read($input);
+              $this->success []= $elem883;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17556,9 +17612,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter863)
+          foreach ($this->success as $iter884)
           {
-            $xfer += $iter863->write($output);
+            $xfer += $iter884->write($output);
           }
         }
         $output->writeListEnd();
@@ -17800,15 +17856,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size864 = 0;
-            $_etype867 = 0;
-            $xfer += $input->readListBegin($_etype867, $_size864);
-            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
+            $_size885 = 0;
+            $_etype888 = 0;
+            $xfer += $input->readListBegin($_etype888, $_size885);
+            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
             {
-              $elem869 = null;
-              $elem869 = new \metastore\FieldSchema();
-              $xfer += $elem869->read($input);
-              $this->success []= $elem869;
+              $elem890 = null;
+              $elem890 = new \metastore\FieldSchema();
+              $xfer += $elem890->read($input);
+              $this->success []= $elem890;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17860,9 +17916,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter870)
+          foreach ($this->success as $iter891)
           {
-            $xfer += $iter870->write($output);
+            $xfer += $iter891->write($output);
           }
         }
         $output->writeListEnd();
@@ -18534,15 +18590,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size871 = 0;
-            $_etype874 = 0;
-            $xfer += $input->readListBegin($_etype874, $_size871);
-            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
+            $_size892 = 0;
+            $_etype895 = 0;
+            $xfer += $input->readListBegin($_etype895, $_size892);
+            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
             {
-              $elem876 = null;
-              $elem876 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem876->read($input);
-              $this->primaryKeys []= $elem876;
+              $elem897 = null;
+              $elem897 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem897->read($input);
+              $this->primaryKeys []= $elem897;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18552,15 +18608,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size877 = 0;
-            $_etype880 = 0;
-            $xfer += $input->readListBegin($_etype880, $_size877);
-            for ($_i881 = 0; $_i881 < $_size877; ++$_i881)
+            $_size898 = 0;
+            $_etype901 = 0;
+            $xfer += $input->readListBegin($_etype901, $_size898);
+            for ($_i902 = 0; $_i902 < $_size898; ++$_i902)
             {
-              $elem882 = null;
-              $elem882 = new \metastore\SQLForeignKey();
-              $xfer += $elem882->read($input);
-              $this->foreignKeys []= $elem882;
+              $elem903 = null;
+              $elem903 = new \metastore\SQLForeignKey();
+              $xfer += $elem903->read($input);
+              $this->foreignKeys []= $elem903;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18570,15 +18626,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem888->read($input);
-              $this->uniqueConstraints []= $elem888;
+              $elem909 = null;
+              $elem909 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem909->read($input);
+              $this->uniqueConstraints []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18588,15 +18644,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size889 = 0;
-            $_etype892 = 0;
-            $xfer += $input->readListBegin($_etype892, $_size889);
-            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
+            $_size910 = 0;
+            $_etype913 = 0;
+            $xfer += $input->readListBegin($_etype913, $_size910);
+            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
             {
-              $elem894 = null;
-              $elem894 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem894->read($input);
-              $this->notNullConstraints []= $elem894;
+              $elem915 = null;
+              $elem915 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem915->read($input);
+              $this->notNullConstraints []= $elem915;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18606,15 +18662,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size895 = 0;
-            $_etype898 = 0;
-            $xfer += $input->readListBegin($_etype898, $_size895);
-            for ($_i899 = 0; $_i899 < $_size895; ++$_i899)
+            $_size916 = 0;
+            $_etype919 = 0;
+            $xfer += $input->readListBegin($_etype919, $_size916);
+            for ($_i920 = 0; $_i920 < $_size916; ++$_i920)
             {
-              $elem900 = null;
-              $elem900 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem900->read($input);
-              $this->defaultConstraints []= $elem900;
+              $elem921 = null;
+              $elem921 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem921->read($input);
+              $this->defaultConstraints []= $elem921;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18624,15 +18680,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size901 = 0;
-            $_etype904 = 0;
-            $xfer += $input->readListBegin($_etype904, $_size901);
-            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
+            $_size922 = 0;
+            $_etype925 = 0;
+            $xfer += $input->readListBegin($_etype925, $_size922);
+            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
             {
-              $elem906 = null;
-              $elem906 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem906->read($input);
-              $this->checkConstraints []= $elem906;
+              $elem927 = null;
+              $elem927 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem927->read($input);
+              $this->checkConstraints []= $elem927;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18668,9 +18724,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter907)
+          foreach ($this->primaryKeys as $iter928)
           {
-            $xfer += $iter907->write($output);
+            $xfer += $iter928->write($output);
           }
         }
         $output->writeListEnd();
@@ -18685,9 +18741,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter908)
+          foreach ($this->foreignKeys as $iter929)
           {
-            $xfer += $iter908->write($output);
+            $xfer += $iter929->write($output);
           }
         }
         $output->writeListEnd();
@@ -18702,9 +18758,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter909)
+          foreach ($this->uniqueConstraints as $iter930)
           {
-            $xfer += $iter909->write($output);
+            $xfer += $iter930->write($output);
           }
         }
         $output->writeListEnd();
@@ -18719,9 +18775,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter910)
+          foreach ($this->notNullConstraints as $iter931)
           {
-            $xfer += $iter910->write($output);
+            $xfer += $iter931->write($output);
           }
         }
         $output->writeListEnd();
@@ -18736,9 +18792,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter911)
+          foreach ($this->defaultConstraints as $iter932)
           {
-            $xfer += $iter911->write($output);
+            $xfer += $iter932->write($output);
           }
         }
         $output->writeListEnd();
@@ -18753,9 +18809,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter912)
+          foreach ($this->checkConstraints as $iter933)
           {
-            $xfer += $iter912->write($output);
+            $xfer += $iter933->write($output);
           }
         }
         $output->writeListEnd();
@@ -20755,14 +20811,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size913 = 0;
-            $_etype916 = 0;
-            $xfer += $input->readListBegin($_etype916, $_size913);
-            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
+            $_size934 = 0;
+            $_etype937 = 0;
+            $xfer += $input->readListBegin($_etype937, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $elem918 = null;
-              $xfer += $input->readString($elem918);
-              $this->partNames []= $elem918;
+              $elem939 = null;
+              $xfer += $input->readString($elem939);
+              $this->partNames []= $elem939;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20800,9 +20856,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter919)
+          foreach ($this->partNames as $iter940)
           {
-            $xfer += $output->writeString($iter919);
+            $xfer += $output->writeString($iter940);
           }
         }
         $output->writeListEnd();
@@ -21053,14 +21109,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size920 = 0;
-            $_etype923 = 0;
-            $xfer += $input->readListBegin($_etype923, $_size920);
-            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
+            $_size941 = 0;
+            $_etype944 = 0;
+            $xfer += $input->readListBegin($_etype944, $_size941);
+            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
             {
-              $elem925 = null;
-              $xfer += $input->readString($elem925);
-              $this->success []= $elem925;
+              $elem946 = null;
+              $xfer += $input->readString($elem946);
+              $this->success []= $elem946;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21096,9 +21152,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter926)
+          foreach ($this->success as $iter947)
           {
-            $xfer += $output->writeString($iter926);
+            $xfer += $output->writeString($iter947);
           }
         }
         $output->writeListEnd();
@@ -21300,14 +21356,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size927 = 0;
-            $_etype930 = 0;
-            $xfer += $input->readListBegin($_etype930, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size948 = 0;
+            $_etype951 = 0;
+            $xfer += $input->readListBegin($_etype951, $_size948);
+            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
             {
-              $elem932 = null;
-              $xfer += $input->readString($elem932);
-              $this->success []= $elem932;
+              $elem953 = null;
+              $xfer += $input->readString($elem953);
+              $this->success []= $elem953;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21343,9 +21399,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter933)
+          foreach ($this->success as $iter954)
           {
-            $xfer += $output->writeString($iter933);
+            $xfer += $output->writeString($iter954);
           }
         }
         $output->writeListEnd();
@@ -21501,14 +21557,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size934 = 0;
-            $_etype937 = 0;
-            $xfer += $input->readListBegin($_etype937, $_size934);
-            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
+            $_size955 = 0;
+            $_etype958 = 0;
+            $xfer += $input->readListBegin($_etype958, $_size955);
+            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
             {
-              $elem939 = null;
-              $xfer += $input->readString($elem939);
-              $this->success []= $elem939;
+              $elem960 = null;
+              $xfer += $input->readString($elem960);
+              $this->success []= $elem960;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21544,9 +21600,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter940)
+          foreach ($this->success as $iter961)
           {
-            $xfer += $output->writeString($iter940);
+            $xfer += $output->writeString($iter961);
           }
         }
         $output->writeListEnd();
@@ -21651,14 +21707,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size941 = 0;
-            $_etype944 = 0;
-            $xfer += $input->readListBegin($_etype944, $_size941);
-            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
+            $_size962 = 0;
+            $_etype965 = 0;
+            $xfer += $input->readListBegin($_etype965, $_size962);
+            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
             {
-              $elem946 = null;
-              $xfer += $input->readString($elem946);
-              $this->tbl_types []= $elem946;
+              $elem967 = null;
+              $xfer += $input->readString($elem967);
+              $this->tbl_types []= $elem967;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21696,9 +21752,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter947)
+          foreach ($this->tbl_types as $iter968)
           {
-            $xfer += $output->writeString($iter947);
+            $xfer += $output->writeString($iter968);
           }
         }
         $output->writeListEnd();
@@ -21775,15 +21831,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size948 = 0;
-            $_etype951 = 0;
-            $xfer += $input->readListBegin($_etype951, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size969 = 0;
+            $_etype972 = 0;
+            $xfer += $input->readListBegin($_etype972, $_size969);
+            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
             {
-              $elem953 = null;
-              $elem953 = new \metastore\TableMeta();
-              $xfer += $elem953->read($input);
-              $this->success []= $elem953;
+              $elem974 = null;
+              $elem974 = new \metastore\TableMeta();
+              $xfer += $elem974->read($input);
+              $this->success []= $elem974;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21819,9 +21875,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter954)
+          foreach ($this->success as $iter975)
           {
-            $xfer += $iter954->write($output);
+            $xfer += $iter975->write($output);
           }
         }
         $output->writeListEnd();
@@ -21977,14 +22033,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size955 = 0;
-            $_etype958 = 0;
-            $xfer += $input->readListBegin($_etype958, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size976 = 0;
+            $_etype979 = 0;
+            $xfer += $input->readListBegin($_etype979, $_size976);
+            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
             {
-              $elem960 = null;
-              $xfer += $input->readString($elem960);
-              $this->success []= $elem960;
+              $elem981 = null;
+              $xfer += $input->readString($elem981);
+              $this->success []= $elem981;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22020,9 +22076,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter961)
+          foreach ($this->success as $iter982)
           {
-            $xfer += $output->writeString($iter961);
+            $xfer += $output->writeString($iter982);
           }
         }
         $output->writeListEnd();
@@ -22337,14 +22393,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size962 = 0;
-            $_etype965 = 0;
-            $xfer += $input->readListBegin($_etype965, $_size962);
-            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
+            $_size983 = 0;
+            $_etype986 = 0;
+            $xfer += $input->readListBegin($_etype986, $_size983);
+            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
             {
-              $elem967 = null;
-              $xfer += $input->readString($elem967);
-              $this->tbl_names []= $elem967;
+              $elem988 = null;
+              $xfer += $input->readString($elem988);
+              $this->tbl_names []= $elem988;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22377,9 +22433,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter968)
+          foreach ($this->tbl_names as $iter989)
           {
-            $xfer += $output->writeString($iter968);
+            $xfer += $output->writeString($iter989);
           }
         }
         $output->writeListEnd();
@@ -22444,15 +22500,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size969 = 0;
-            $_etype972 = 0;
-            $xfer += $input->readListBegin($_etype972, $_size969);
-            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
+            $_size990 = 0;
+            $_etype993 = 0;
+            $xfer += $input->readListBegin($_etype993, $_size990);
+            for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
             {
-              $elem974 = null;
-              $elem974 = new \metastore\Table();
-              $xfer += $elem974->read($input);
-              $this->success []= $elem974;
+              $elem995 = null;
+              $elem995 = new \metastore\Table();
+              $xfer += $elem995->read($input);
+              $this->success []= $elem995;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22480,9 +22536,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter975)
+          foreach ($this->success as $iter996)
           {
-            $xfer += $iter975->write($output);
+            $xfer += $iter996->write($output);
           }
         }
         $output->writeListEnd();
@@ -23009,14 +23065,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size976 = 0;
-            $_etype979 = 0;
-            $xfer += $input->readListBegin($_etype979, $_size976);
-            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
+            $_size997 = 0;
+            $_etype1000 = 0;
+            $xfer += $input->readListBegin($_etype1000, $_size997);
+            for ($_i1001 = 0; $_i1001 < $_size997; ++$_i1001)
             {
-              $elem981 = null;
-              $xfer += $input->readString($elem981);
-              $this->tbl_names []= $elem981;
+              $elem1002 = null;
+              $xfer += $input->readString($elem1002);
+              $this->tbl_names []= $elem1002;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23049,9 +23105,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter982)
+          foreach ($this->tbl_names as $iter1003)
           {
-            $xfer += $output->writeString($iter982);
+            $xfer += $output->writeString($iter1003);
           }
         }
         $output->writeListEnd();
@@ -23156,18 +23212,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size983 = 0;
-            $_ktype984 = 0;
-            $_vtype985 = 0;
-            $xfer += $input->readMapBegin($_ktype984, $_vtype985, $_size983);
-            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
+            $_size1004 = 0;
+            $_ktype1005 = 0;
+            $_vtype1006 = 0;
+            $xfer += $input->readMapBegin($_ktype1005, $_vtype1006, $_size1004);
+            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
             {
-              $key988 = '';
-              $val989 = new \metastore\Materialization();
-              $xfer += $input->readString($key988);
-              $val989 = new \metastore\Materialization();
-              $xfer += $val989->read($input);
-              $this->success[$key988] = $val989;
+              $key1009 = '';
+              $val1010 = new \metastore\Materialization();
+              $xfer += $input->readString($key1009);
+              $val1010 = new \metastore\Materialization();
+              $xfer += $val1010->read($input);
+              $this->success[$key1009] = $val1010;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -23219,10 +23275,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter990 => $viter991)
+          foreach ($this->success as $kiter1011 => $viter1012)
           {
-            $xfer += $output->writeString($kiter990);
-            $xfer += $viter991->write($output);
+            $xfer += $output->writeString($kiter1011);
+            $xfer += $viter1012->write($output);
           }
         }
         $output->writeMapEnd();
@@ -23734,14 +23790,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size1013 = 0;
+            $_etype1016 = 0;
+            $xfer += $input->readListBegin($_etype1016, $_size1013);
+            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
             {
-              $elem997 = null;
-              $xfer += $input->readString($elem997);
-              $this->success []= $elem997;
+              $elem1018 = null;
+              $xfer += $input->readString($elem1018);
+              $this->success []= $elem1018;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23793,9 +23849,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter998)
+          foreach ($this->success as $iter1019)
           {
-            $xfer += $output->writeString($iter998);
+            $xfer += $output->writeString($iter1019);
           }
         }
         $output->writeListEnd();
@@ -25108,15 +25164,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size1020 = 0;
+            $_etype1023 = 0;
+            $xfer += $input->readListBegin($_etype1023, $_size1020);
+            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
             {
-              $elem1004 = null;
-              $elem1004 = new \metastore\Partition();
-              $xfer += $elem1004->read($input);
-              $this->new_parts []= $elem1004;
+              $elem1025 = null;
+              $elem1025 = new \metastore\Partition();
+              $xfer += $elem1025->read($input);
+              $this->new_parts []= $elem1025;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25144,9 +25200,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1005)
+          foreach ($this->new_parts as $iter1026)
           {
-            $xfer += $iter1005->write($output);
+            $xfer += $iter1026->write($output);
           }
         }
         $output->writeListEnd();
@@ -25361,15 +25417,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
+            $_size1027 = 0;
+            $_etype1030 = 0;
+            $xfer += $input->readListBegin($_etype1030, $_size1027);
+            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
             {
-              $elem1011 = null;
-              $elem1011 = new \metastore\PartitionSpec();
-              $xfer += $elem1011->read($input);
-              $this->new_parts []= $elem1011;
+              $elem1032 = null;
+              $elem1032 = new \metastore\PartitionSpec();
+              $xfer += $elem1032->read($input);
+              $this->new_parts []= $elem1032;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25397,9 +25453,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1012)
+          foreach ($this->new_parts as $iter1033)
           {
-            $xfer += $iter1012->write($output);
+            $xfer += $iter1033->write($output);
           }
         }
         $output->writeListEnd();
@@ -25649,14 +25705,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size1034 = 0;
+            $_etype1037 = 0;
+            $xfer += $input->readListBegin($_etype1037, $_size1034);
+            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->part_vals []= $elem1018;
+              $elem1039 = null;
+              $xfer += $input->readString($elem1039);
+              $this->part_vals []= $elem1039;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25694,9 +25750,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1019)
+          foreach ($this->part_vals as $iter1040)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter1040);
           }
         }
         $output->writeListEnd();
@@ -26198,14 +26254,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1020 = 0;
-            $_etype1023 = 0;
-            $xfer += $input->readListBegin($_etype1023, $_size1020);
-            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
+            $_size1041 = 0;
+            $_etype1044 = 0;
+            $xfer += $input->readListBegin($_etype1044, $_size1041);
+            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
             {
-              $elem1025 = null;
-              $xfer += $input->readString($elem1025);
-              $this->part_vals []= $elem1025;
+              $elem1046 = null;
+              $xfer += $input->readString($elem1046);
+              $this->part_vals []= $elem1046;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26251,9 +26307,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1026)
+          foreach ($this->part_vals as $iter1047)
           {
-            $xfer += $output->writeString($iter1026);
+            $xfer += $output->writeString($iter1047);
           }
         }
         $output->writeListEnd();
@@ -27107,14 +27163,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1048 = 0;
+            $_etype1051 = 0;
+            $xfer += $input->readListBegin($_etype1051, $_size1048);
+            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
             {
-              $elem1032 = null;
-              $xfer += $input->readString($elem1032);
-              $this->part_vals []= $elem1032;
+              $elem1053 = null;
+              $xfer += $input->readString($elem1053);
+              $this->part_vals []= $elem1053;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27159,9 +27215,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1033)
+          foreach ($this->part_vals as $iter1054)
           {
-            $xfer += $output->writeString($iter1033);
+            $xfer += $output->writeString($iter1054);
           }
         }
         $output->writeListEnd();
@@ -27414,14 +27470,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1034 = 0;
-            $_etype1037 = 0;
-            $xfer += $input->readListBegin($_etype1037, $_size1034);
-            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
+            $_size1055 = 0;
+            $_etype1058 = 0;
+            $xfer += $input->readListBegin($_etype1058, $_size1055);
+            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
             {
-              $elem1039 = null;
-              $xfer += $input->readString($elem1039);
-              $this->part_vals []= $elem1039;
+              $elem1060 = null;
+              $xfer += $input->readString($elem1060);
+              $this->part_vals []= $elem1060;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27474,9 +27530,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1040)
+          foreach ($this->part_vals as $iter1061)
           {
-            $xfer += $output->writeString($iter1040);
+            $xfer += $output->writeString($iter1061);
           }
         }
         $output->writeListEnd();
@@ -28490,14 +28546,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1041 = 0;
-            $_etype1044 = 0;
-            $xfer += $input->readListBegin($_etype1044, $_size1041);
-            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
+            $_size1062 = 0;
+            $_etype1065 = 0;
+            $xfer += $input->readListBegin($_etype1065, $_size1062);
+            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
             {
-              $elem1046 = null;
-              $xfer += $input->readString($elem1046);
-              $this->part_vals []= $elem1046;
+              $elem1067 = null;
+              $xfer += $input->readString($elem1067);
+              $this->part_vals []= $elem1067;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28535,9 +28591,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1047)
+          foreach ($this->part_vals as $iter1068)
           {
-            $xfer += $output->writeString($iter1047);
+            $xfer += $output->writeString($iter1068);
           }
         }
         $output->writeListEnd();
@@ -28779,17 +28835,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1048 = 0;
-            $_ktype1049 = 0;
-            $_vtype1050 = 0;
-            $xfer += $input->readMapBegin($_ktype1049, $_vtype1050, $_size1048);
-            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
+            $_size1069 = 0;
+            $_ktype1070 = 0;
+            $_vtype1071 = 0;
+            $xfer += $input->readMapBegin($_ktype1070, $_vtype1071, $_size1069);
+            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
             {
-              $key1053 = '';
-              $val1054 = '';
-              $xfer += $input->readString($key1053);
-              $xfer += $input->readString($val1054);
-              $this->partitionSpecs[$key1053] = $val1054;
+              $key1074 = '';
+              $val1075 = '';
+              $xfer += $input->readString($key1074);
+              $xfer += $input->readString($val1075);
+              $this->partitionSpecs[$key1074] = $val1075;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28845,10 +28901,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1055 => $viter1056)
+          foreach ($this->partitionSpecs as $kiter1076 => $viter1077)
           {
-            $xfer += $output->writeString($kiter1055);
-            $xfer += $output->writeString($viter1056);
+            $xfer += $output->writeString($kiter1076);
+            $xfer += $output->writeString($viter1077);
           }
         }
         $output->writeMapEnd();
@@ -29160,17 +29216,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1057 = 0;
-            $_ktype1058 = 0;
-            $_vtype1059 = 0;
-            $xfer += $input->readMapBegin($_ktype1058, $_vtype1059, $_size1057);
-            for ($_i1061 = 0; $_i1061 < $_size1057; ++$_i1061)
+            $_size1078 = 0;
+            $_ktype1079 = 0;
+            $_vtype1080 = 0;
+            $xfer += $input->readMapBegin($_ktype1079, $_vtype1080, $_size1078);
+            for ($_i1082 = 0; $_i1082 < $_size1078; ++$_i1082)
             {
-              $key1062 = '';
-              $val1063 = '';
-              $xfer += $input->readString($key1062);
-              $xfer += $input->readString($val1063);
-              $this->partitionSpecs[$key1062] = $val1063;
+              $key1083 = '';
+              $val1084 = '';
+              $xfer += $input->readString($key1083);
+              $xfer += $input->readString($val1084);
+              $this->partitionSpecs[$key1083] = $val1084;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29226,10 +29282,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1064 => $viter1065)
+          foreach ($this->partitionSpecs as $kiter1085 => $viter1086)
           {
-            $xfer += $output->writeString($kiter1064);
-            $xfer += $output->writeString($viter1065);
+            $xfer += $output->writeString($kiter1085);
+            $xfer += $output->writeString($viter1086);
           }
         }
         $output->writeMapEnd();
@@ -29362,15 +29418,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1066 = 0;
-            $_etype1069 = 0;
-            $xfer += $input->readListBegin($_etype1069, $_size1066);
-            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1071 = null;
-              $elem1071 = new \metastore\Partition();
-              $xfer += $elem1071->read($input);
-              $this->success []= $elem1071;
+              $elem1092 = null;
+              $elem1092 = new \metastore\Partition();
+              $xfer += $elem1092->read($input);
+              $this->success []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29430,9 +29486,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1072)
+          foreach ($this->success as $iter1093)
           {
-            $xfer += $iter1072->write($output);
+            $xfer += $iter1093->write($output);
           }
         }
         $output->writeListEnd();
@@ -29578,14 +29634,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1073 = 0;
-            $_etype1076 = 0;
-            $xfer += $input->readListBegin($_etype1076, $_size1073);
-            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
+            $_size1094 = 0;
+            $_etype1097 = 0;
+            $xfer += $input->readListBegin($_etype1097, $_size1094);
+            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
             {
-              $elem1078 = null;
-              $xfer += $input->readString($elem1078);
-              $this->part_vals []= $elem1078;
+              $elem1099 = null;
+              $xfer += $input->readString($elem1099);
+              $this->part_vals []= $elem1099;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29602,14 +29658,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1079 = 0;
-            $_etype1082 = 0;
-            $xfer += $input->readListBegin($_etype1082, $_size1079);
-            for ($_i1083 = 0; $_i1083 < $_size1079; ++$_i1083)
+            $_size1100 = 0;
+            $_etype1103 = 0;
+            $xfer += $input->readListBegin($_etype1103, $_size1100);
+            for ($_i1104 = 0; $_i1104 < $_size1100; ++$_i1104)
             {
-              $elem1084 = null;
-              $xfer += $input->readString($elem1084);
-              $this->group_names []= $elem1084;
+              $elem1105 = null;
+              $xfer += $input->readString($elem1105);
+              $this->group_names []= $elem1105;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29647,9 +29703,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1085)
+          foreach ($this->part_vals as $iter1106)
           {
-            $xfer += $output->writeString($iter1085);
+            $xfer += $output->writeString($iter1106);
           }
         }
         $output->writeListEnd();
@@ -29669,9 +29725,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1086)
+          foreach ($this->group_names as $iter1107)
           {
-            $xfer += $output->writeString($iter1086);
+            $xfer += $output->writeString($iter1107);
           }
         }
         $output->writeListEnd();
@@ -30262,15 +30318,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1087 = 0;
-            $_etype1090 = 0;
-            $xfer += $input->readListBegin($_etype1090, $_size1087);
-            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
+            $_size1108 = 0;
+            $_etype1111 = 0;
+            $xfer += $input->readListBegin($_etype1111, $_size1108);
+            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
             {
-              $elem1092 = null;
-              $elem1092 = new \metastore\Partition();
-              $xfer += $elem1092->read($input);
-              $this->success []= $elem1092;
+              $elem1113 = null;
+              $elem1113 = new \metastore\Partition();
+              $xfer += $elem1113->read($input);
+              $this->success []= $elem1113;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30314,9 +30370,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1093)
+          foreach ($this->success as $iter1114)
           {
-            $xfer += $iter1093->write($output);
+            $xfer += $iter1114->write($output);
           }
         }
         $output->writeListEnd();
@@ -30462,14 +30518,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1094 = 0;
-            $_etype1097 = 0;
-            $xfer += $input->readListBegin($_etype1097, $_size1094);
-            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
+            $_size1115 = 0;
+            $_etype1118 = 0;
+            $xfer += $input->readListBegin($_etype1118, $_size1115);
+            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
             {
-              $elem1099 = null;
-              $xfer += $input->readString($elem1099);
-              $this->group_names []= $elem1099;
+              $elem1120 = null;
+              $xfer += $input->readString($elem1120);
+              $this->group_names []= $elem1120;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30517,9 +30573,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1100)
+          foreach ($this->group_names as $iter1121)
           {
-            $xfer += $output->writeString($iter1100);
+            $xfer += $output->writeString($iter1121);
           }
         }
         $output->writeListEnd();
@@ -30608,15 +30664,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1101 = 0;
-            $_etype1104 = 0;
-            $xfer += $input->readListBegin($_etype1104, $_size1101);
-            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
+            $_size1122 = 0;
+            $_etype1125 = 0;
+            $xfer += $input->readListBegin($_etype1125, $_size1122);
+            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
             {
-              $elem1106 = null;
-              $elem1106 = new \metastore\Partition();
-              $xfer += $elem1106->read($input);
-              $this->success []= $elem1106;
+              $elem1127 = null;
+              $elem1127 = new \metastore\Partition();
+              $xfer += $elem1127->read($input);
+              $this->success []= $elem1127;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30660,9 +30716,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1107)
+          foreach ($this->success as $iter1128)
           {
-            $xfer += $iter1107->write($output);
+            $xfer += $iter1128->write($output);
           }
         }
         $output->writeListEnd();
@@ -30882,15 +30938,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1108 = 0;
-            $_etype1111 = 0;
-            $xfer += $input->readListBegin($_etype1111, $_size1108);
-            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
+            $_size1129 = 0;
+            $_etype1132 = 0;
+            $xfer += $input->readListBegin($_etype1132, $_size1129);
+            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
             {
-              $elem1113 = null;
-              $elem1113 = new \metastore\PartitionSpec();
-              $xfer += $elem1113->read($input);
-              $this->success []= $elem1113;
+              $elem1134 = null;
+              $elem1134 = new \metastore\PartitionSpec();
+              $xfer += $elem1134->read($input);
+              $this->success []= $elem1134;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30934,9 +30990,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1114)
+          foreach ($this->success as $iter1135)
           {
-            $xfer += $iter1114->write($output);
+            $xfer += $iter1135->write($output);
           }
         }
         $output->writeListEnd();
@@ -31155,14 +31211,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1115 = 0;
-            $_etype1118 = 0;
-            $xfer += $input->readListBegin($_etype1118, $_size1115);
-            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
+            $_size1136 = 0;
+            $_etype1139 = 0;
+            $xfer += $input->readListBegin($_etype1139, $_size1136);
+            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
             {
-              $elem1120 = null;
-              $xfer += $input->readString($elem1120);
-              $this->success []= $elem1120;
+              $elem1141 = null;
+              $xfer += $input->readString($elem1141);
+              $this->success []= $elem1141;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31206,9 +31262,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1121)
+          foreach ($this->success as $iter1142)
           {
-            $xfer += $output->writeString($iter1121);
+            $xfer += $output->writeString($iter1142);
           }
         }
         $output->writeListEnd();
@@ -31539,14 +31595,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1122 = 0;
-            $_etype1125 = 0;
-            $xfer += $input->readListBegin($_etype1125, $_size1122);
-            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
+            $_size1143 = 0;
+            $_etype1146 = 0;
+            $xfer += $input->readListBegin($_etype1146, $_size1143);
+            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
             {
-              $elem1127 = null;
-              $xfer += $input->readString($elem1127);
-              $this->part_vals []= $elem1127;
+              $elem1148 = null;
+              $xfer += $input->readString($elem1148);
+              $this->part_vals []= $elem1148;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31591,9 +31647,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1128)
+          foreach ($this->part_vals as $iter1149)
           {
-            $xfer += $output->writeString($iter1128);
+            $xfer += $output->writeString($iter1149);
           }
         }
         $output->writeListEnd();
@@ -31687,15 +31743,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1129 = 0;
-            $_etype1132 = 0;
-            $xfer += $input->readListBegin($_etype1132, $_size1129);
-            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
+            $_size1150 = 0;
+            $_etype1153 = 0;
+            $xfer += $input->readListBegin($_etype1153, $_size1150);
+            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
             {
-              $elem1134 = null;
-              $elem1134 = new \metastore\Partition();
-              $xfer += $elem1134->read($input);
-              $this->success []= $elem1134;
+              $elem1155 = null;
+              $elem1155 = new \metastore\Partition();
+              $xfer += $elem1155->read($input);
+              $this->success []= $elem1155;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31739,9 +31795,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1135)
+          foreach ($this->success as $iter1156)
           {
-            $xfer += $iter1135->write($output);
+            $xfer += $iter1156->write($output);
           }
         }
         $output->writeListEnd();
@@ -31888,14 +31944,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1136 = 0;
-            $_etype1139 = 0;
-            $xfer += $input->readListBegin($_etype1139, $_size1136);
-            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
+            $_size1157 = 0;
+            $_etype1160 = 0;
+            $xfer += $input->readListBegin($_etype1160, $_size1157);
+            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
             {
-              $elem1141 = null;
-              $xfer += $input->readString($elem1141);
-              $this->part_vals []= $elem1141;
+              $elem1162 = null;
+              $xfer += $input->readString($elem1162);
+              $this->part_vals []= $elem1162;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31919,14 +31975,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1142 = 0;
-            $_etype1145 = 0;
-            $xfer += $input->readListBegin($_etype1145, $_size1142);
-            for ($_i1146 = 0; $_i1146 < $_size1142; ++$_i1146)
+            $_size1163 = 0;
+            $_etype1166 = 0;
+            $xfer += $input->readListBegin($_etype1166, $_size1163);
+            for ($_i1167 = 0; $_i1167 < $_size1163; ++$_i1167)
             {
-              $elem1147 = null;
-              $xfer += $input->readString($elem1147);
-              $this->group_names []= $elem1147;
+              $elem1168 = null;
+              $xfer += $input->readString($elem1168);
+              $this->group_names []= $elem1168;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31964,9 +32020,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1148)
+          foreach ($this->part_vals as $iter1169)
           {
-            $xfer += $output->writeString($iter1148);
+            $xfer += $output->writeString($iter1169);
           }
         }
         $output->writeListEnd();
@@ -31991,9 +32047,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1149)
+          foreach ($this->group_names as $iter1170)
           {
-            $xfer += $output->writeString($iter1149);
+            $xfer += $output->writeString($iter1170);
           }
         }
         $output->writeListEnd();
@@ -32082,15 +32138,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1150 = 0;
-            $_etype1153 = 0;
-            $xfer += $input->readListBegin($_etype1153, $_size1150);
-            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
+            $_size1171 = 0;
+            $_etype1174 = 0;
+            $xfer += $input->readListBegin($_etype1174, $_size1171);
+            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
             {
-              $elem1155 = null;
-              $elem1155 = new \metastore\Partition();
-              $xfer += $elem1155->read($input);
-              $this->success []= $elem1155;
+              $elem1176 = null;
+              $elem1176 = new \metastore\Partition();
+              $xfer += $elem1176->read($input);
+              $this->success []= $elem1176;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32134,9 +32190,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1156)
+          foreach ($this->success as $iter1177)
           {
-            $xfer += $iter1156->write($output);
+            $xfer += $iter1177->write($output);
           }
         }
         $output->writeListEnd();
@@ -32257,14 +32313,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1157 = 0;
-            $_etype1160 = 0;
-            $xfer += $input->readListBegin($_etype1160, $_size1157);
-            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
+            $_size1178 = 0;
+            $_etype1181 = 0;
+            $xfer += $input->readListBegin($_etype1181, $_size1178);
+            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
             {
-              $elem1162 = null;
-              $xfer += $input->readString($elem1162);
-              $this->part_vals []= $elem1162;
+              $elem1183 = null;
+              $xfer += $input->readString($elem1183);
+              $this->part_vals []= $elem1183;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32309,9 +32365,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1163)
+          foreach ($this->part_vals as $iter1184)
           {
-            $xfer += $output->writeString($iter1163);
+            $xfer += $output->writeString($iter1184);
           }
         }
         $output->writeListEnd();
@@ -32404,14 +32460,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1164 = 0;
-            $_etype1167 = 0;
-            $xfer += $input->readListBegin($_etype1167, $_size1164);
-            for ($_i1168 = 0; $_i1168 < $_size1164; ++$_i1168)
+            $_size1185 = 0;
+            $_etype1188 = 0;
+            $xfer += $input->readListBegin($_etype1188, $_size1185);
+            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
             {
-              $elem1169 = null;
-              $xfer += $input->readString($elem1169);
-              $this->success []= $elem1169;
+              $elem1190 = null;
+              $xfer += $input->readString($elem1190);
+              $this->success []= $elem1190;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32455,9 +32511,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1170)
+          foreach ($this->success as $iter1191)
           {
-            $xfer += $output->writeString($iter1170);
+            $xfer += $output->writeString($iter1191);
           }
         }
         $output->writeListEnd();
@@ -32700,15 +32756,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1171 = 0;
-            $_etype1174 = 0;
-            $xfer += $input->readListBegin($_etype1174, $_size1171);
-            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
+            $_size1192 = 0;
+            $_etype1195 = 0;
+            $xfer += $input->readListBegin($_etype1195, $_size1192);
+            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
             {
-              $elem1176 = null;
-              $elem1176 = new \metastore\Partition();
-              $xfer += $elem1176->read($input);
-              $this->success []= $elem1176;
+              $elem1197 = null;
+              $elem1197 = new \metastore\Partition();
+              $xfer += $elem1197->read($input);
+              $this->success []= $elem1197;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32752,9 +32808,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1177)
+          foreach ($this->success as $iter1198)
           {
-            $xfer += $iter1177->write($output);
+            $xfer += $iter1198->write($output);
           }
         }
         $output->writeListEnd();
@@ -32997,15 +33053,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1178 = 0;
-            $_etype1181 = 0;
-            $xfer += $input->readListBegin($_etype1181, $_size1178);
-            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
+            $_size1199 = 0;
+            $_etype1202 = 0;
+            $xfer += $input->readListBegin($_etype1202, $_size1199);
+            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
             {
-              $elem1183 = null;
-              $elem1183 = new \metastore\PartitionSpec();
-              $xfer += $elem1183->read($input);
-              $this->success []= $elem1183;
+              $elem1204 = null;
+              $elem1204 = new \metastore\PartitionSpec();
+              $xfer += $elem1204->read($input);
+              $this->success []= $elem1204;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33049,9 +33105,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1184)
+          foreach ($this->success as $iter1205)
           {
-            $xfer += $iter1184->write($output);
+            $xfer += $iter1205->write($output);
           }
         }
         $output->writeListEnd();
@@ -33617,14 +33673,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1185 = 0;
-            $_etype1188 = 0;
-            $xfer += $input->readListBegin($_etype1188, $_size1185);
-            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
+            $_size1206 = 0;
+            $_etype1209 = 0;
+            $xfer += $input->readListBegin($_etype1209, $_size1206);
+            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
             {
-              $elem1190 = null;
-              $xfer += $input->readString($elem1190);
-              $this->names []= $elem1190;
+              $elem1211 = null;
+              $xfer += $input->readString($elem1211);
+              $this->names []= $elem1211;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33662,9 +33718,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1191)
+          foreach ($this->names as $iter1212)
           {
-            $xfer += $output->writeString($iter1191);
+            $xfer += $output->writeString($iter1212);
           }
         }
         $output->writeListEnd();
@@ -33753,15 +33809,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1192 = 0;
-            $_etype1195 = 0;
-            $xfer += $input->readListBegin($_etype1195, $_size1192);
-            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
+            $_size1213 = 0;
+            $_etype1216 = 0;
+            $xfer += $input->readListBegin($_etype1216, $_size1213);
+            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
             {
-              $elem1197 = null;
-              $elem1197 = new \metastore\Partition();
-              $xfer += $elem1197->read($input);
-              $this->success []= $elem1197;
+              $elem1218 = null;
+              $elem1218 = new \metastore\Partition();
+              $xfer += $elem1218->read($input);
+              $this->success []= $elem1218;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33805,9 +33861,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1198)
+          foreach ($this->success as $iter1219)
           {
-            $xfer += $iter1198->write($output);
+            $xfer += $iter1219->write($output);
           }
         }
         $output->writeListEnd();
@@ -34146,15 +34202,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1199 = 0;
-            $_etype1202 = 0;
-            $xfer += $input->readListBegin($_etype1202, $_size1199);
-            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
+            $_size1220 = 0;
+            $_etype1223 = 0;
+            $xfer += $input->readListBegin($_etype1223, $_size1220);
+            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
             {
-              $elem1204 = null;
-              $elem1204 = new \metastore\Partition();
-              $xfer += $elem1204->read($input);
-              $this->new_parts []= $elem1204;
+              $elem1225 = null;
+              $elem1225 = new \metastore\Partition();
+              $xfer += $elem1225->read($input);
+              $this->new_parts []= $elem1225;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34192,9 +34248,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1205)
+          foreach ($this->new_parts as $iter1226)
           {
-            $xfer += $iter1205->write($output);
+            $xfer += $iter1226->write($output);
           }
         }
         $output->writeListEnd();
@@ -34409,15 +34465,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1206 = 0;
-            $_etype1209 = 0;
-            $xfer += $input->readListBegin($_etype1209, $_size1206);
-            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
+            $_size1227 = 0;
+            $_etype1230 = 0;
+            $xfer += $input->readListBegin($_etype1230, $_size1227);
+            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
             {
-              $elem1211 = null;
-              $elem1211 = new \metastore\Partition();
-              $xfer += $elem1211->read($input);
-              $this->new_parts []= $elem1211;
+              $elem1232 = null;
+              $elem1232 = new \metastore\Partition();
+              $xfer += $elem1232->read($input);
+              $this->new_parts []= $elem1232;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34463,9 +34519,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1212)
+          foreach ($this->new_parts as $iter1233)
           {
-            $xfer += $iter1212->write($output);
+            $xfer += $iter1233->write($output);
           }
         }
         $output->writeListEnd();
@@ -34943,14 +34999,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1213 = 0;
-            $_etype1216 = 0;
-            $xfer += $input->readListBegin($_etype1216, $_size1213);
-            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
+            $_size1234 = 0;
+            $_etype1237 = 0;
+            $xfer += $input->readListBegin($_etype1237, $_size1234);
+            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
             {
-              $elem1218 = null;
-              $xfer += $input->readString($elem1218);
-              $this->part_vals []= $elem1218;
+              $elem1239 = null;
+              $xfer += $input->readString($elem1239);
+              $this->part_vals []= $elem1239;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34996,9 +35052,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1219)
+          foreach ($this->part_vals as $iter1240)
           {
-            $xfer += $output->writeString($iter1219);
+            $xfer += $output->writeString($iter1240);
           }
         }
         $output->writeListEnd();
@@ -35183,14 +35239,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1220 = 0;
-            $_etype1223 = 0;
-            $xfer += $input->readListBegin($_etype1223, $_size1220);
-            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
+            $_size1241 = 0;
+            $_etype1244 = 0;
+            $xfer += $input->readListBegin($_etype1244, $_size1241);
+            for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245)
             {
-              $elem1225 = null;
-              $xfer += $input->readString($elem1225);
-              $this->part_vals []= $elem1225;
+              $elem1246 = null;
+              $xfer += $input->readString($elem1246);
+              $this->part_vals []= $elem1246;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35225,9 +35281,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1226)
+          foreach ($this->part_vals as $iter1247)
           {
-            $xfer += $output->writeString($iter1226);
+            $xfer += $output->writeString($iter1247);
           }
         }
         $output->writeListEnd();
@@ -35681,14 +35737,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1227 = 0;
-            $_etype1230 = 0;
-            $xfer += $input->readListBegin($_etype1230, $_size1227);
-            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
+            $_size1248 = 0;
+            $_etype1251 = 0;
+            $xfer += $input->readListBegin($_etype1251, $_size1248);
+            for ($_i1252 = 0; $_i1252 < $_size1248; ++$_i1252)
             {
-              $elem1232 = null;
-              $xfer += $input->readString($elem1232);
-              $this->success []= $elem1232;
+              $elem1253 = null;
+              $xfer += $input->readString($elem1253);
+              $this->success []= $elem1253;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35724,9 +35780,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1233)
+          foreach ($this->success as $iter1254)
           {
-            $xfer += $output->writeString($iter1233);
+            $xfer += $output->writeString($iter1254);
           }
         }
         $output->writeListEnd();
@@ -35886,17 +35942,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1234 = 0;
-            $_ktype1235 = 0;
-            $_vtype1236 = 0;
-            $xfer += $input->readMapBegin($_ktype1235, $_vtype1236, $_size1234);
-            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
+            $_size1255 = 0;
+            $_ktype1256 = 0;
+            $_vtype1257 = 0;
+            $xfer += $input->readMapBegin($_ktype1256, $_vtype1257, $_size1255);
+            for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259)
             {
-              $key1239 = '';
-              $val1240 = '';
-              $xfer += $input->readString($key1239);
-              $xfer += $input->readString($val1240);
-              $this->success[$key1239] = $val1240;
+              $key1260 = '';
+              $val1261 = '';
+              $xfer += $input->readString($key1260);
+              $xfer += $input->readString($val1261);
+              $this->success[$key1260] = $val1261;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -35932,10 +35988,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1241 => $viter1242)
+          foreach ($this->success as $kiter1262 => $viter1263)
           {
-            $xfer += $output->writeString($kiter1241);
-            $xfer += $output->writeString($viter1242);
+            $xfer += $output->writeString($kiter1262);
+            $xfer += $output->writeString($viter1263);
           }
         }
         $output->writeMapEnd();
@@ -36055,17 +36111,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1243 = 0;

<TRUNCATED>

[14/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index e459bc2..a816ae7 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2334,14 +2334,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1202;
-            ::apache::thrift::protocol::TType _etype1205;
-            xfer += iprot->readListBegin(_etype1205, _size1202);
-            this->success.resize(_size1202);
-            uint32_t _i1206;
-            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
+            uint32_t _size1226;
+            ::apache::thrift::protocol::TType _etype1229;
+            xfer += iprot->readListBegin(_etype1229, _size1226);
+            this->success.resize(_size1226);
+            uint32_t _i1230;
+            for (_i1230 = 0; _i1230 < _size1226; ++_i1230)
             {
-              xfer += iprot->readString(this->success[_i1206]);
+              xfer += iprot->readString(this->success[_i1230]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2380,10 +2380,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 _iter1207;
-      for (_iter1207 = this->success.begin(); _iter1207 != this->success.end(); ++_iter1207)
+      std::vector<std::string> ::const_iterator _iter1231;
+      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
       {
-        xfer += oprot->writeString((*_iter1207));
+        xfer += oprot->writeString((*_iter1231));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2428,14 +2428,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1208;
-            ::apache::thrift::protocol::TType _etype1211;
-            xfer += iprot->readListBegin(_etype1211, _size1208);
-            (*(this->success)).resize(_size1208);
-            uint32_t _i1212;
-            for (_i1212 = 0; _i1212 < _size1208; ++_i1212)
+            uint32_t _size1232;
+            ::apache::thrift::protocol::TType _etype1235;
+            xfer += iprot->readListBegin(_etype1235, _size1232);
+            (*(this->success)).resize(_size1232);
+            uint32_t _i1236;
+            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
             {
-              xfer += iprot->readString((*(this->success))[_i1212]);
+              xfer += iprot->readString((*(this->success))[_i1236]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2552,14 +2552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1213;
-            ::apache::thrift::protocol::TType _etype1216;
-            xfer += iprot->readListBegin(_etype1216, _size1213);
-            this->success.resize(_size1213);
-            uint32_t _i1217;
-            for (_i1217 = 0; _i1217 < _size1213; ++_i1217)
+            uint32_t _size1237;
+            ::apache::thrift::protocol::TType _etype1240;
+            xfer += iprot->readListBegin(_etype1240, _size1237);
+            this->success.resize(_size1237);
+            uint32_t _i1241;
+            for (_i1241 = 0; _i1241 < _size1237; ++_i1241)
             {
-              xfer += iprot->readString(this->success[_i1217]);
+              xfer += iprot->readString(this->success[_i1241]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2598,10 +2598,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 _iter1218;
-      for (_iter1218 = this->success.begin(); _iter1218 != this->success.end(); ++_iter1218)
+      std::vector<std::string> ::const_iterator _iter1242;
+      for (_iter1242 = this->success.begin(); _iter1242 != this->success.end(); ++_iter1242)
       {
-        xfer += oprot->writeString((*_iter1218));
+        xfer += oprot->writeString((*_iter1242));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2646,14 +2646,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1219;
-            ::apache::thrift::protocol::TType _etype1222;
-            xfer += iprot->readListBegin(_etype1222, _size1219);
-            (*(this->success)).resize(_size1219);
-            uint32_t _i1223;
-            for (_i1223 = 0; _i1223 < _size1219; ++_i1223)
+            uint32_t _size1243;
+            ::apache::thrift::protocol::TType _etype1246;
+            xfer += iprot->readListBegin(_etype1246, _size1243);
+            (*(this->success)).resize(_size1243);
+            uint32_t _i1247;
+            for (_i1247 = 0; _i1247 < _size1243; ++_i1247)
             {
-              xfer += iprot->readString((*(this->success))[_i1223]);
+              xfer += iprot->readString((*(this->success))[_i1247]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3715,17 +3715,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1224;
-            ::apache::thrift::protocol::TType _ktype1225;
-            ::apache::thrift::protocol::TType _vtype1226;
-            xfer += iprot->readMapBegin(_ktype1225, _vtype1226, _size1224);
-            uint32_t _i1228;
-            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
+            uint32_t _size1248;
+            ::apache::thrift::protocol::TType _ktype1249;
+            ::apache::thrift::protocol::TType _vtype1250;
+            xfer += iprot->readMapBegin(_ktype1249, _vtype1250, _size1248);
+            uint32_t _i1252;
+            for (_i1252 = 0; _i1252 < _size1248; ++_i1252)
             {
-              std::string _key1229;
-              xfer += iprot->readString(_key1229);
-              Type& _val1230 = this->success[_key1229];
-              xfer += _val1230.read(iprot);
+              std::string _key1253;
+              xfer += iprot->readString(_key1253);
+              Type& _val1254 = this->success[_key1253];
+              xfer += _val1254.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3764,11 +3764,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 _iter1231;
-      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
+      std::map<std::string, Type> ::const_iterator _iter1255;
+      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
       {
-        xfer += oprot->writeString(_iter1231->first);
-        xfer += _iter1231->second.write(oprot);
+        xfer += oprot->writeString(_iter1255->first);
+        xfer += _iter1255->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3813,17 +3813,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1232;
-            ::apache::thrift::protocol::TType _ktype1233;
-            ::apache::thrift::protocol::TType _vtype1234;
-            xfer += iprot->readMapBegin(_ktype1233, _vtype1234, _size1232);
-            uint32_t _i1236;
-            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
+            uint32_t _size1256;
+            ::apache::thrift::protocol::TType _ktype1257;
+            ::apache::thrift::protocol::TType _vtype1258;
+            xfer += iprot->readMapBegin(_ktype1257, _vtype1258, _size1256);
+            uint32_t _i1260;
+            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
             {
-              std::string _key1237;
-              xfer += iprot->readString(_key1237);
-              Type& _val1238 = (*(this->success))[_key1237];
-              xfer += _val1238.read(iprot);
+              std::string _key1261;
+              xfer += iprot->readString(_key1261);
+              Type& _val1262 = (*(this->success))[_key1261];
+              xfer += _val1262.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3977,14 +3977,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1239;
-            ::apache::thrift::protocol::TType _etype1242;
-            xfer += iprot->readListBegin(_etype1242, _size1239);
-            this->success.resize(_size1239);
-            uint32_t _i1243;
-            for (_i1243 = 0; _i1243 < _size1239; ++_i1243)
+            uint32_t _size1263;
+            ::apache::thrift::protocol::TType _etype1266;
+            xfer += iprot->readListBegin(_etype1266, _size1263);
+            this->success.resize(_size1263);
+            uint32_t _i1267;
+            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
             {
-              xfer += this->success[_i1243].read(iprot);
+              xfer += this->success[_i1267].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4039,10 +4039,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 _iter1244;
-      for (_iter1244 = this->success.begin(); _iter1244 != this->success.end(); ++_iter1244)
+      std::vector<FieldSchema> ::const_iterator _iter1268;
+      for (_iter1268 = this->success.begin(); _iter1268 != this->success.end(); ++_iter1268)
       {
-        xfer += (*_iter1244).write(oprot);
+        xfer += (*_iter1268).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4095,14 +4095,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1245;
-            ::apache::thrift::protocol::TType _etype1248;
-            xfer += iprot->readListBegin(_etype1248, _size1245);
-            (*(this->success)).resize(_size1245);
-            uint32_t _i1249;
-            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
+            uint32_t _size1269;
+            ::apache::thrift::protocol::TType _etype1272;
+            xfer += iprot->readListBegin(_etype1272, _size1269);
+            (*(this->success)).resize(_size1269);
+            uint32_t _i1273;
+            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
             {
-              xfer += (*(this->success))[_i1249].read(iprot);
+              xfer += (*(this->success))[_i1273].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4288,14 +4288,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1250;
-            ::apache::thrift::protocol::TType _etype1253;
-            xfer += iprot->readListBegin(_etype1253, _size1250);
-            this->success.resize(_size1250);
-            uint32_t _i1254;
-            for (_i1254 = 0; _i1254 < _size1250; ++_i1254)
+            uint32_t _size1274;
+            ::apache::thrift::protocol::TType _etype1277;
+            xfer += iprot->readListBegin(_etype1277, _size1274);
+            this->success.resize(_size1274);
+            uint32_t _i1278;
+            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
             {
-              xfer += this->success[_i1254].read(iprot);
+              xfer += this->success[_i1278].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4350,10 +4350,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 _iter1255;
-      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
+      std::vector<FieldSchema> ::const_iterator _iter1279;
+      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
       {
-        xfer += (*_iter1255).write(oprot);
+        xfer += (*_iter1279).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4406,14 +4406,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1256;
-            ::apache::thrift::protocol::TType _etype1259;
-            xfer += iprot->readListBegin(_etype1259, _size1256);
-            (*(this->success)).resize(_size1256);
-            uint32_t _i1260;
-            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
+            uint32_t _size1280;
+            ::apache::thrift::protocol::TType _etype1283;
+            xfer += iprot->readListBegin(_etype1283, _size1280);
+            (*(this->success)).resize(_size1280);
+            uint32_t _i1284;
+            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
             {
-              xfer += (*(this->success))[_i1260].read(iprot);
+              xfer += (*(this->success))[_i1284].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4583,14 +4583,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1261;
-            ::apache::thrift::protocol::TType _etype1264;
-            xfer += iprot->readListBegin(_etype1264, _size1261);
-            this->success.resize(_size1261);
-            uint32_t _i1265;
-            for (_i1265 = 0; _i1265 < _size1261; ++_i1265)
+            uint32_t _size1285;
+            ::apache::thrift::protocol::TType _etype1288;
+            xfer += iprot->readListBegin(_etype1288, _size1285);
+            this->success.resize(_size1285);
+            uint32_t _i1289;
+            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
             {
-              xfer += this->success[_i1265].read(iprot);
+              xfer += this->success[_i1289].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4645,10 +4645,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 _iter1266;
-      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
+      std::vector<FieldSchema> ::const_iterator _iter1290;
+      for (_iter1290 = this->success.begin(); _iter1290 != this->success.end(); ++_iter1290)
       {
-        xfer += (*_iter1266).write(oprot);
+        xfer += (*_iter1290).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4701,14 +4701,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1267;
-            ::apache::thrift::protocol::TType _etype1270;
-            xfer += iprot->readListBegin(_etype1270, _size1267);
-            (*(this->success)).resize(_size1267);
-            uint32_t _i1271;
-            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
+            uint32_t _size1291;
+            ::apache::thrift::protocol::TType _etype1294;
+            xfer += iprot->readListBegin(_etype1294, _size1291);
+            (*(this->success)).resize(_size1291);
+            uint32_t _i1295;
+            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
             {
-              xfer += (*(this->success))[_i1271].read(iprot);
+              xfer += (*(this->success))[_i1295].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4894,14 +4894,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1272;
-            ::apache::thrift::protocol::TType _etype1275;
-            xfer += iprot->readListBegin(_etype1275, _size1272);
-            this->success.resize(_size1272);
-            uint32_t _i1276;
-            for (_i1276 = 0; _i1276 < _size1272; ++_i1276)
+            uint32_t _size1296;
+            ::apache::thrift::protocol::TType _etype1299;
+            xfer += iprot->readListBegin(_etype1299, _size1296);
+            this->success.resize(_size1296);
+            uint32_t _i1300;
+            for (_i1300 = 0; _i1300 < _size1296; ++_i1300)
             {
-              xfer += this->success[_i1276].read(iprot);
+              xfer += this->success[_i1300].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4956,10 +4956,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 _iter1277;
-      for (_iter1277 = this->success.begin(); _iter1277 != this->success.end(); ++_iter1277)
+      std::vector<FieldSchema> ::const_iterator _iter1301;
+      for (_iter1301 = this->success.begin(); _iter1301 != this->success.end(); ++_iter1301)
       {
-        xfer += (*_iter1277).write(oprot);
+        xfer += (*_iter1301).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5012,14 +5012,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1278;
-            ::apache::thrift::protocol::TType _etype1281;
-            xfer += iprot->readListBegin(_etype1281, _size1278);
-            (*(this->success)).resize(_size1278);
-            uint32_t _i1282;
-            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
+            uint32_t _size1302;
+            ::apache::thrift::protocol::TType _etype1305;
+            xfer += iprot->readListBegin(_etype1305, _size1302);
+            (*(this->success)).resize(_size1302);
+            uint32_t _i1306;
+            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
             {
-              xfer += (*(this->success))[_i1282].read(iprot);
+              xfer += (*(this->success))[_i1306].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5612,14 +5612,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1283;
-            ::apache::thrift::protocol::TType _etype1286;
-            xfer += iprot->readListBegin(_etype1286, _size1283);
-            this->primaryKeys.resize(_size1283);
-            uint32_t _i1287;
-            for (_i1287 = 0; _i1287 < _size1283; ++_i1287)
+            uint32_t _size1307;
+            ::apache::thrift::protocol::TType _etype1310;
+            xfer += iprot->readListBegin(_etype1310, _size1307);
+            this->primaryKeys.resize(_size1307);
+            uint32_t _i1311;
+            for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
             {
-              xfer += this->primaryKeys[_i1287].read(iprot);
+              xfer += this->primaryKeys[_i1311].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5632,14 +5632,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1288;
-            ::apache::thrift::protocol::TType _etype1291;
-            xfer += iprot->readListBegin(_etype1291, _size1288);
-            this->foreignKeys.resize(_size1288);
-            uint32_t _i1292;
-            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
+            uint32_t _size1312;
+            ::apache::thrift::protocol::TType _etype1315;
+            xfer += iprot->readListBegin(_etype1315, _size1312);
+            this->foreignKeys.resize(_size1312);
+            uint32_t _i1316;
+            for (_i1316 = 0; _i1316 < _size1312; ++_i1316)
             {
-              xfer += this->foreignKeys[_i1292].read(iprot);
+              xfer += this->foreignKeys[_i1316].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5652,14 +5652,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1293;
-            ::apache::thrift::protocol::TType _etype1296;
-            xfer += iprot->readListBegin(_etype1296, _size1293);
-            this->uniqueConstraints.resize(_size1293);
-            uint32_t _i1297;
-            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
+            uint32_t _size1317;
+            ::apache::thrift::protocol::TType _etype1320;
+            xfer += iprot->readListBegin(_etype1320, _size1317);
+            this->uniqueConstraints.resize(_size1317);
+            uint32_t _i1321;
+            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
             {
-              xfer += this->uniqueConstraints[_i1297].read(iprot);
+              xfer += this->uniqueConstraints[_i1321].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5672,14 +5672,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1298;
-            ::apache::thrift::protocol::TType _etype1301;
-            xfer += iprot->readListBegin(_etype1301, _size1298);
-            this->notNullConstraints.resize(_size1298);
-            uint32_t _i1302;
-            for (_i1302 = 0; _i1302 < _size1298; ++_i1302)
+            uint32_t _size1322;
+            ::apache::thrift::protocol::TType _etype1325;
+            xfer += iprot->readListBegin(_etype1325, _size1322);
+            this->notNullConstraints.resize(_size1322);
+            uint32_t _i1326;
+            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
             {
-              xfer += this->notNullConstraints[_i1302].read(iprot);
+              xfer += this->notNullConstraints[_i1326].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5692,14 +5692,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1303;
-            ::apache::thrift::protocol::TType _etype1306;
-            xfer += iprot->readListBegin(_etype1306, _size1303);
-            this->defaultConstraints.resize(_size1303);
-            uint32_t _i1307;
-            for (_i1307 = 0; _i1307 < _size1303; ++_i1307)
+            uint32_t _size1327;
+            ::apache::thrift::protocol::TType _etype1330;
+            xfer += iprot->readListBegin(_etype1330, _size1327);
+            this->defaultConstraints.resize(_size1327);
+            uint32_t _i1331;
+            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
             {
-              xfer += this->defaultConstraints[_i1307].read(iprot);
+              xfer += this->defaultConstraints[_i1331].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5712,14 +5712,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1308;
-            ::apache::thrift::protocol::TType _etype1311;
-            xfer += iprot->readListBegin(_etype1311, _size1308);
-            this->checkConstraints.resize(_size1308);
-            uint32_t _i1312;
-            for (_i1312 = 0; _i1312 < _size1308; ++_i1312)
+            uint32_t _size1332;
+            ::apache::thrift::protocol::TType _etype1335;
+            xfer += iprot->readListBegin(_etype1335, _size1332);
+            this->checkConstraints.resize(_size1332);
+            uint32_t _i1336;
+            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
             {
-              xfer += this->checkConstraints[_i1312].read(iprot);
+              xfer += this->checkConstraints[_i1336].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5752,10 +5752,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1313;
-    for (_iter1313 = this->primaryKeys.begin(); _iter1313 != this->primaryKeys.end(); ++_iter1313)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1337;
+    for (_iter1337 = this->primaryKeys.begin(); _iter1337 != this->primaryKeys.end(); ++_iter1337)
     {
-      xfer += (*_iter1313).write(oprot);
+      xfer += (*_iter1337).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5764,10 +5764,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1314;
-    for (_iter1314 = this->foreignKeys.begin(); _iter1314 != this->foreignKeys.end(); ++_iter1314)
+    std::vector<SQLForeignKey> ::const_iterator _iter1338;
+    for (_iter1338 = this->foreignKeys.begin(); _iter1338 != this->foreignKeys.end(); ++_iter1338)
     {
-      xfer += (*_iter1314).write(oprot);
+      xfer += (*_iter1338).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5776,10 +5776,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1315;
-    for (_iter1315 = this->uniqueConstraints.begin(); _iter1315 != this->uniqueConstraints.end(); ++_iter1315)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1339;
+    for (_iter1339 = this->uniqueConstraints.begin(); _iter1339 != this->uniqueConstraints.end(); ++_iter1339)
     {
-      xfer += (*_iter1315).write(oprot);
+      xfer += (*_iter1339).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5788,10 +5788,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1316;
-    for (_iter1316 = this->notNullConstraints.begin(); _iter1316 != this->notNullConstraints.end(); ++_iter1316)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1340;
+    for (_iter1340 = this->notNullConstraints.begin(); _iter1340 != this->notNullConstraints.end(); ++_iter1340)
     {
-      xfer += (*_iter1316).write(oprot);
+      xfer += (*_iter1340).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5800,10 +5800,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1317;
-    for (_iter1317 = this->defaultConstraints.begin(); _iter1317 != this->defaultConstraints.end(); ++_iter1317)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1341;
+    for (_iter1341 = this->defaultConstraints.begin(); _iter1341 != this->defaultConstraints.end(); ++_iter1341)
     {
-      xfer += (*_iter1317).write(oprot);
+      xfer += (*_iter1341).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5812,10 +5812,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1318;
-    for (_iter1318 = this->checkConstraints.begin(); _iter1318 != this->checkConstraints.end(); ++_iter1318)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1342;
+    for (_iter1342 = this->checkConstraints.begin(); _iter1342 != this->checkConstraints.end(); ++_iter1342)
     {
-      xfer += (*_iter1318).write(oprot);
+      xfer += (*_iter1342).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5843,10 +5843,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1319;
-    for (_iter1319 = (*(this->primaryKeys)).begin(); _iter1319 != (*(this->primaryKeys)).end(); ++_iter1319)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1343;
+    for (_iter1343 = (*(this->primaryKeys)).begin(); _iter1343 != (*(this->primaryKeys)).end(); ++_iter1343)
     {
-      xfer += (*_iter1319).write(oprot);
+      xfer += (*_iter1343).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5855,10 +5855,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1320;
-    for (_iter1320 = (*(this->foreignKeys)).begin(); _iter1320 != (*(this->foreignKeys)).end(); ++_iter1320)
+    std::vector<SQLForeignKey> ::const_iterator _iter1344;
+    for (_iter1344 = (*(this->foreignKeys)).begin(); _iter1344 != (*(this->foreignKeys)).end(); ++_iter1344)
     {
-      xfer += (*_iter1320).write(oprot);
+      xfer += (*_iter1344).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5867,10 +5867,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1321;
-    for (_iter1321 = (*(this->uniqueConstraints)).begin(); _iter1321 != (*(this->uniqueConstraints)).end(); ++_iter1321)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1345;
+    for (_iter1345 = (*(this->uniqueConstraints)).begin(); _iter1345 != (*(this->uniqueConstraints)).end(); ++_iter1345)
     {
-      xfer += (*_iter1321).write(oprot);
+      xfer += (*_iter1345).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5879,10 +5879,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1322;
-    for (_iter1322 = (*(this->notNullConstraints)).begin(); _iter1322 != (*(this->notNullConstraints)).end(); ++_iter1322)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1346;
+    for (_iter1346 = (*(this->notNullConstraints)).begin(); _iter1346 != (*(this->notNullConstraints)).end(); ++_iter1346)
     {
-      xfer += (*_iter1322).write(oprot);
+      xfer += (*_iter1346).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5891,10 +5891,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1323;
-    for (_iter1323 = (*(this->defaultConstraints)).begin(); _iter1323 != (*(this->defaultConstraints)).end(); ++_iter1323)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1347;
+    for (_iter1347 = (*(this->defaultConstraints)).begin(); _iter1347 != (*(this->defaultConstraints)).end(); ++_iter1347)
     {
-      xfer += (*_iter1323).write(oprot);
+      xfer += (*_iter1347).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5903,10 +5903,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1324;
-    for (_iter1324 = (*(this->checkConstraints)).begin(); _iter1324 != (*(this->checkConstraints)).end(); ++_iter1324)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1348;
+    for (_iter1348 = (*(this->checkConstraints)).begin(); _iter1348 != (*(this->checkConstraints)).end(); ++_iter1348)
     {
-      xfer += (*_iter1324).write(oprot);
+      xfer += (*_iter1348).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8074,14 +8074,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1325;
-            ::apache::thrift::protocol::TType _etype1328;
-            xfer += iprot->readListBegin(_etype1328, _size1325);
-            this->partNames.resize(_size1325);
-            uint32_t _i1329;
-            for (_i1329 = 0; _i1329 < _size1325; ++_i1329)
+            uint32_t _size1349;
+            ::apache::thrift::protocol::TType _etype1352;
+            xfer += iprot->readListBegin(_etype1352, _size1349);
+            this->partNames.resize(_size1349);
+            uint32_t _i1353;
+            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
             {
-              xfer += iprot->readString(this->partNames[_i1329]);
+              xfer += iprot->readString(this->partNames[_i1353]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8118,10 +8118,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1330;
-    for (_iter1330 = this->partNames.begin(); _iter1330 != this->partNames.end(); ++_iter1330)
+    std::vector<std::string> ::const_iterator _iter1354;
+    for (_iter1354 = this->partNames.begin(); _iter1354 != this->partNames.end(); ++_iter1354)
     {
-      xfer += oprot->writeString((*_iter1330));
+      xfer += oprot->writeString((*_iter1354));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8153,10 +8153,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1331;
-    for (_iter1331 = (*(this->partNames)).begin(); _iter1331 != (*(this->partNames)).end(); ++_iter1331)
+    std::vector<std::string> ::const_iterator _iter1355;
+    for (_iter1355 = (*(this->partNames)).begin(); _iter1355 != (*(this->partNames)).end(); ++_iter1355)
     {
-      xfer += oprot->writeString((*_iter1331));
+      xfer += oprot->writeString((*_iter1355));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8400,14 +8400,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1332;
-            ::apache::thrift::protocol::TType _etype1335;
-            xfer += iprot->readListBegin(_etype1335, _size1332);
-            this->success.resize(_size1332);
-            uint32_t _i1336;
-            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
+            uint32_t _size1356;
+            ::apache::thrift::protocol::TType _etype1359;
+            xfer += iprot->readListBegin(_etype1359, _size1356);
+            this->success.resize(_size1356);
+            uint32_t _i1360;
+            for (_i1360 = 0; _i1360 < _size1356; ++_i1360)
             {
-              xfer += iprot->readString(this->success[_i1336]);
+              xfer += iprot->readString(this->success[_i1360]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8446,10 +8446,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 _iter1337;
-      for (_iter1337 = this->success.begin(); _iter1337 != this->success.end(); ++_iter1337)
+      std::vector<std::string> ::const_iterator _iter1361;
+      for (_iter1361 = this->success.begin(); _iter1361 != this->success.end(); ++_iter1361)
       {
-        xfer += oprot->writeString((*_iter1337));
+        xfer += oprot->writeString((*_iter1361));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8494,14 +8494,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1338;
-            ::apache::thrift::protocol::TType _etype1341;
-            xfer += iprot->readListBegin(_etype1341, _size1338);
-            (*(this->success)).resize(_size1338);
-            uint32_t _i1342;
-            for (_i1342 = 0; _i1342 < _size1338; ++_i1342)
+            uint32_t _size1362;
+            ::apache::thrift::protocol::TType _etype1365;
+            xfer += iprot->readListBegin(_etype1365, _size1362);
+            (*(this->success)).resize(_size1362);
+            uint32_t _i1366;
+            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
             {
-              xfer += iprot->readString((*(this->success))[_i1342]);
+              xfer += iprot->readString((*(this->success))[_i1366]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8671,14 +8671,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1343;
-            ::apache::thrift::protocol::TType _etype1346;
-            xfer += iprot->readListBegin(_etype1346, _size1343);
-            this->success.resize(_size1343);
-            uint32_t _i1347;
-            for (_i1347 = 0; _i1347 < _size1343; ++_i1347)
+            uint32_t _size1367;
+            ::apache::thrift::protocol::TType _etype1370;
+            xfer += iprot->readListBegin(_etype1370, _size1367);
+            this->success.resize(_size1367);
+            uint32_t _i1371;
+            for (_i1371 = 0; _i1371 < _size1367; ++_i1371)
             {
-              xfer += iprot->readString(this->success[_i1347]);
+              xfer += iprot->readString(this->success[_i1371]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8717,10 +8717,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1348;
-      for (_iter1348 = this->success.begin(); _iter1348 != this->success.end(); ++_iter1348)
+      std::vector<std::string> ::const_iterator _iter1372;
+      for (_iter1372 = this->success.begin(); _iter1372 != this->success.end(); ++_iter1372)
       {
-        xfer += oprot->writeString((*_iter1348));
+        xfer += oprot->writeString((*_iter1372));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8765,14 +8765,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1349;
-            ::apache::thrift::protocol::TType _etype1352;
-            xfer += iprot->readListBegin(_etype1352, _size1349);
-            (*(this->success)).resize(_size1349);
-            uint32_t _i1353;
-            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
+            uint32_t _size1373;
+            ::apache::thrift::protocol::TType _etype1376;
+            xfer += iprot->readListBegin(_etype1376, _size1373);
+            (*(this->success)).resize(_size1373);
+            uint32_t _i1377;
+            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
             {
-              xfer += iprot->readString((*(this->success))[_i1353]);
+              xfer += iprot->readString((*(this->success))[_i1377]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8910,14 +8910,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1354;
-            ::apache::thrift::protocol::TType _etype1357;
-            xfer += iprot->readListBegin(_etype1357, _size1354);
-            this->success.resize(_size1354);
-            uint32_t _i1358;
-            for (_i1358 = 0; _i1358 < _size1354; ++_i1358)
+            uint32_t _size1378;
+            ::apache::thrift::protocol::TType _etype1381;
+            xfer += iprot->readListBegin(_etype1381, _size1378);
+            this->success.resize(_size1378);
+            uint32_t _i1382;
+            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
             {
-              xfer += iprot->readString(this->success[_i1358]);
+              xfer += iprot->readString(this->success[_i1382]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8956,10 +8956,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     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 _iter1359;
-      for (_iter1359 = this->success.begin(); _iter1359 != this->success.end(); ++_iter1359)
+      std::vector<std::string> ::const_iterator _iter1383;
+      for (_iter1383 = this->success.begin(); _iter1383 != this->success.end(); ++_iter1383)
       {
-        xfer += oprot->writeString((*_iter1359));
+        xfer += oprot->writeString((*_iter1383));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9004,14 +9004,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1360;
-            ::apache::thrift::protocol::TType _etype1363;
-            xfer += iprot->readListBegin(_etype1363, _size1360);
-            (*(this->success)).resize(_size1360);
-            uint32_t _i1364;
-            for (_i1364 = 0; _i1364 < _size1360; ++_i1364)
+            uint32_t _size1384;
+            ::apache::thrift::protocol::TType _etype1387;
+            xfer += iprot->readListBegin(_etype1387, _size1384);
+            (*(this->success)).resize(_size1384);
+            uint32_t _i1388;
+            for (_i1388 = 0; _i1388 < _size1384; ++_i1388)
             {
-              xfer += iprot->readString((*(this->success))[_i1364]);
+              xfer += iprot->readString((*(this->success))[_i1388]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9086,14 +9086,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size1365;
-            ::apache::thrift::protocol::TType _etype1368;
-            xfer += iprot->readListBegin(_etype1368, _size1365);
-            this->tbl_types.resize(_size1365);
-            uint32_t _i1369;
-            for (_i1369 = 0; _i1369 < _size1365; ++_i1369)
+            uint32_t _size1389;
+            ::apache::thrift::protocol::TType _etype1392;
+            xfer += iprot->readListBegin(_etype1392, _size1389);
+            this->tbl_types.resize(_size1389);
+            uint32_t _i1393;
+            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
             {
-              xfer += iprot->readString(this->tbl_types[_i1369]);
+              xfer += iprot->readString(this->tbl_types[_i1393]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9130,10 +9130,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter1370;
-    for (_iter1370 = this->tbl_types.begin(); _iter1370 != this->tbl_types.end(); ++_iter1370)
+    std::vector<std::string> ::const_iterator _iter1394;
+    for (_iter1394 = this->tbl_types.begin(); _iter1394 != this->tbl_types.end(); ++_iter1394)
     {
-      xfer += oprot->writeString((*_iter1370));
+      xfer += oprot->writeString((*_iter1394));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9165,10 +9165,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter1371;
-    for (_iter1371 = (*(this->tbl_types)).begin(); _iter1371 != (*(this->tbl_types)).end(); ++_iter1371)
+    std::vector<std::string> ::const_iterator _iter1395;
+    for (_iter1395 = (*(this->tbl_types)).begin(); _iter1395 != (*(this->tbl_types)).end(); ++_iter1395)
     {
-      xfer += oprot->writeString((*_iter1371));
+      xfer += oprot->writeString((*_iter1395));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9209,14 +9209,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1372;
-            ::apache::thrift::protocol::TType _etype1375;
-            xfer += iprot->readListBegin(_etype1375, _size1372);
-            this->success.resize(_size1372);
-            uint32_t _i1376;
-            for (_i1376 = 0; _i1376 < _size1372; ++_i1376)
+            uint32_t _size1396;
+            ::apache::thrift::protocol::TType _etype1399;
+            xfer += iprot->readListBegin(_etype1399, _size1396);
+            this->success.resize(_size1396);
+            uint32_t _i1400;
+            for (_i1400 = 0; _i1400 < _size1396; ++_i1400)
             {
-              xfer += this->success[_i1376].read(iprot);
+              xfer += this->success[_i1400].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9255,10 +9255,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter1377;
-      for (_iter1377 = this->success.begin(); _iter1377 != this->success.end(); ++_iter1377)
+      std::vector<TableMeta> ::const_iterator _iter1401;
+      for (_iter1401 = this->success.begin(); _iter1401 != this->success.end(); ++_iter1401)
       {
-        xfer += (*_iter1377).write(oprot);
+        xfer += (*_iter1401).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9303,14 +9303,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1378;
-            ::apache::thrift::protocol::TType _etype1381;
-            xfer += iprot->readListBegin(_etype1381, _size1378);
-            (*(this->success)).resize(_size1378);
-            uint32_t _i1382;
-            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
+            uint32_t _size1402;
+            ::apache::thrift::protocol::TType _etype1405;
+            xfer += iprot->readListBegin(_etype1405, _size1402);
+            (*(this->success)).resize(_size1402);
+            uint32_t _i1406;
+            for (_i1406 = 0; _i1406 < _size1402; ++_i1406)
             {
-              xfer += (*(this->success))[_i1382].read(iprot);
+              xfer += (*(this->success))[_i1406].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9448,14 +9448,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1383;
-            ::apache::thrift::protocol::TType _etype1386;
-            xfer += iprot->readListBegin(_etype1386, _size1383);
-            this->success.resize(_size1383);
-            uint32_t _i1387;
-            for (_i1387 = 0; _i1387 < _size1383; ++_i1387)
+            uint32_t _size1407;
+            ::apache::thrift::protocol::TType _etype1410;
+            xfer += iprot->readListBegin(_etype1410, _size1407);
+            this->success.resize(_size1407);
+            uint32_t _i1411;
+            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
             {
-              xfer += iprot->readString(this->success[_i1387]);
+              xfer += iprot->readString(this->success[_i1411]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9494,10 +9494,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 _iter1388;
-      for (_iter1388 = this->success.begin(); _iter1388 != this->success.end(); ++_iter1388)
+      std::vector<std::string> ::const_iterator _iter1412;
+      for (_iter1412 = this->success.begin(); _iter1412 != this->success.end(); ++_iter1412)
       {
-        xfer += oprot->writeString((*_iter1388));
+        xfer += oprot->writeString((*_iter1412));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9542,14 +9542,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1389;
-            ::apache::thrift::protocol::TType _etype1392;
-            xfer += iprot->readListBegin(_etype1392, _size1389);
-            (*(this->success)).resize(_size1389);
-            uint32_t _i1393;
-            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
+            uint32_t _size1413;
+            ::apache::thrift::protocol::TType _etype1416;
+            xfer += iprot->readListBegin(_etype1416, _size1413);
+            (*(this->success)).resize(_size1413);
+            uint32_t _i1417;
+            for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
             {
-              xfer += iprot->readString((*(this->success))[_i1393]);
+              xfer += iprot->readString((*(this->success))[_i1417]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9859,14 +9859,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 _size1394;
-            ::apache::thrift::protocol::TType _etype1397;
-            xfer += iprot->readListBegin(_etype1397, _size1394);
-            this->tbl_names.resize(_size1394);
-            uint32_t _i1398;
-            for (_i1398 = 0; _i1398 < _size1394; ++_i1398)
+            uint32_t _size1418;
+            ::apache::thrift::protocol::TType _etype1421;
+            xfer += iprot->readListBegin(_etype1421, _size1418);
+            this->tbl_names.resize(_size1418);
+            uint32_t _i1422;
+            for (_i1422 = 0; _i1422 < _size1418; ++_i1422)
             {
-              xfer += iprot->readString(this->tbl_names[_i1398]);
+              xfer += iprot->readString(this->tbl_names[_i1422]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9899,10 +9899,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 _iter1399;
-    for (_iter1399 = this->tbl_names.begin(); _iter1399 != this->tbl_names.end(); ++_iter1399)
+    std::vector<std::string> ::const_iterator _iter1423;
+    for (_iter1423 = this->tbl_names.begin(); _iter1423 != this->tbl_names.end(); ++_iter1423)
     {
-      xfer += oprot->writeString((*_iter1399));
+      xfer += oprot->writeString((*_iter1423));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9930,10 +9930,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 _iter1400;
-    for (_iter1400 = (*(this->tbl_names)).begin(); _iter1400 != (*(this->tbl_names)).end(); ++_iter1400)
+    std::vector<std::string> ::const_iterator _iter1424;
+    for (_iter1424 = (*(this->tbl_names)).begin(); _iter1424 != (*(this->tbl_names)).end(); ++_iter1424)
     {
-      xfer += oprot->writeString((*_iter1400));
+      xfer += oprot->writeString((*_iter1424));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9974,14 +9974,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 _size1401;
-            ::apache::thrift::protocol::TType _etype1404;
-            xfer += iprot->readListBegin(_etype1404, _size1401);
-            this->success.resize(_size1401);
-            uint32_t _i1405;
-            for (_i1405 = 0; _i1405 < _size1401; ++_i1405)
+            uint32_t _size1425;
+            ::apache::thrift::protocol::TType _etype1428;
+            xfer += iprot->readListBegin(_etype1428, _size1425);
+            this->success.resize(_size1425);
+            uint32_t _i1429;
+            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
             {
-              xfer += this->success[_i1405].read(iprot);
+              xfer += this->success[_i1429].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10012,10 +10012,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 _iter1406;
-      for (_iter1406 = this->success.begin(); _iter1406 != this->success.end(); ++_iter1406)
+      std::vector<Table> ::const_iterator _iter1430;
+      for (_iter1430 = this->success.begin(); _iter1430 != this->success.end(); ++_iter1430)
       {
-        xfer += (*_iter1406).write(oprot);
+        xfer += (*_iter1430).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10056,14 +10056,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 _size1407;
-            ::apache::thrift::protocol::TType _etype1410;
-            xfer += iprot->readListBegin(_etype1410, _size1407);
-            (*(this->success)).resize(_size1407);
-            uint32_t _i1411;
-            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
+            uint32_t _size1431;
+            ::apache::thrift::protocol::TType _etype1434;
+            xfer += iprot->readListBegin(_etype1434, _size1431);
+            (*(this->success)).resize(_size1431);
+            uint32_t _i1435;
+            for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
             {
-              xfer += (*(this->success))[_i1411].read(iprot);
+              xfer += (*(this->success))[_i1435].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10596,14 +10596,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1412;
-            ::apache::thrift::protocol::TType _etype1415;
-            xfer += iprot->readListBegin(_etype1415, _size1412);
-            this->tbl_names.resize(_size1412);
-            uint32_t _i1416;
-            for (_i1416 = 0; _i1416 < _size1412; ++_i1416)
+            uint32_t _size1436;
+            ::apache::thrift::protocol::TType _etype1439;
+            xfer += iprot->readListBegin(_etype1439, _size1436);
+            this->tbl_names.resize(_size1436);
+            uint32_t _i1440;
+            for (_i1440 = 0; _i1440 < _size1436; ++_i1440)
             {
-              xfer += iprot->readString(this->tbl_names[_i1416]);
+              xfer += iprot->readString(this->tbl_names[_i1440]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10636,10 +10636,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   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 _iter1417;
-    for (_iter1417 = this->tbl_names.begin(); _iter1417 != this->tbl_names.end(); ++_iter1417)
+    std::vector<std::string> ::const_iterator _iter1441;
+    for (_iter1441 = this->tbl_names.begin(); _iter1441 != this->tbl_names.end(); ++_iter1441)
     {
-      xfer += oprot->writeString((*_iter1417));
+      xfer += oprot->writeString((*_iter1441));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10667,10 +10667,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   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 _iter1418;
-    for (_iter1418 = (*(this->tbl_names)).begin(); _iter1418 != (*(this->tbl_names)).end(); ++_iter1418)
+    std::vector<std::string> ::const_iterator _iter1442;
+    for (_iter1442 = (*(this->tbl_names)).begin(); _iter1442 != (*(this->tbl_names)).end(); ++_iter1442)
     {
-      xfer += oprot->writeString((*_iter1418));
+      xfer += oprot->writeString((*_iter1442));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10711,17 +10711,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1419;
-            ::apache::thrift::protocol::TType _ktype1420;
-            ::apache::thrift::protocol::TType _vtype1421;
-            xfer += iprot->readMapBegin(_ktype1420, _vtype1421, _size1419);
-            uint32_t _i1423;
-            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
+            uint32_t _size1443;
+            ::apache::thrift::protocol::TType _ktype1444;
+            ::apache::thrift::protocol::TType _vtype1445;
+            xfer += iprot->readMapBegin(_ktype1444, _vtype1445, _size1443);
+            uint32_t _i1447;
+            for (_i1447 = 0; _i1447 < _size1443; ++_i1447)
             {
-              std::string _key1424;
-              xfer += iprot->readString(_key1424);
-              Materialization& _val1425 = this->success[_key1424];
-              xfer += _val1425.read(iprot);
+              std::string _key1448;
+              xfer += iprot->readString(_key1448);
+              Materialization& _val1449 = this->success[_key1448];
+              xfer += _val1449.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10776,11 +10776,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     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, Materialization> ::const_iterator _iter1426;
-      for (_iter1426 = this->success.begin(); _iter1426 != this->success.end(); ++_iter1426)
+      std::map<std::string, Materialization> ::const_iterator _iter1450;
+      for (_iter1450 = this->success.begin(); _iter1450 != this->success.end(); ++_iter1450)
       {
-        xfer += oprot->writeString(_iter1426->first);
-        xfer += _iter1426->second.write(oprot);
+        xfer += oprot->writeString(_iter1450->first);
+        xfer += _iter1450->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -10833,17 +10833,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1427;
-            ::apache::thrift::protocol::TType _ktype1428;
-            ::apache::thrift::protocol::TType _vtype1429;
-            xfer += iprot->readMapBegin(_ktype1428, _vtype1429, _size1427);
-            uint32_t _i1431;
-            for (_i1431 = 0; _i1431 < _size1427; ++_i1431)
+            uint32_t _size1451;
+            ::apache::thrift::protocol::TType _ktype1452;
+            ::apache::thrift::protocol::TType _vtype1453;
+            xfer += iprot->readMapBegin(_ktype1452, _vtype1453, _size1451);
+            uint32_t _i1455;
+            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
             {
-              std::string _key1432;
-              xfer += iprot->readString(_key1432);
-              Materialization& _val1433 = (*(this->success))[_key1432];
-              xfer += _val1433.read(iprot);
+              std::string _key1456;
+              xfer += iprot->readString(_key1456);
+              Materialization& _val1457 = (*(this->success))[_key1456];
+              xfer += _val1457.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11304,14 +11304,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 _size1434;
-            ::apache::thrift::protocol::TType _etype1437;
-            xfer += iprot->readListBegin(_etype1437, _size1434);
-            this->success.resize(_size1434);
-            uint32_t _i1438;
-            for (_i1438 = 0; _i1438 < _size1434; ++_i1438)
+            uint32_t _size1458;
+            ::apache::thrift::protocol::TType _etype1461;
+            xfer += iprot->readListBegin(_etype1461, _size1458);
+            this->success.resize(_size1458);
+            uint32_t _i1462;
+            for (_i1462 = 0; _i1462 < _size1458; ++_i1462)
             {
-              xfer += iprot->readString(this->success[_i1438]);
+              xfer += iprot->readString(this->success[_i1462]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11366,10 +11366,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 _iter1439;
-      for (_iter1439 = this->success.begin(); _iter1439 != this->success.end(); ++_iter1439)
+      std::vector<std::string> ::const_iterator _iter1463;
+      for (_iter1463 = this->success.begin(); _iter1463 != this->success.end(); ++_iter1463)
       {
-        xfer += oprot->writeString((*_iter1439));
+        xfer += oprot->writeString((*_iter1463));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11422,14 +11422,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 _size1440;
-            ::apache::thrift::protocol::TType _etype1443;
-            xfer += iprot->readListBegin(_etype1443, _size1440);
-            (*(this->success)).resize(_size1440);
-            uint32_t _i1444;
-            for (_i1444 = 0; _i1444 < _size1440; ++_i1444)
+            uint32_t _size1464;
+            ::apache::thrift::protocol::TType _etype1467;
+            xfer += iprot->readListBegin(_etype1467, _size1464);
+            (*(this->success)).resize(_size1464);
+            uint32_t _i1468;
+            for (_i1468 = 0; _i1468 < _size1464; ++_i1468)
             {
-              xfer += iprot->readString((*(this->success))[_i1444]);
+              xfer += iprot->readString((*(this->success))[_i1468]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12763,14 +12763,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1445;
-            ::apache::thrift::protocol::TType _etype1448;
-            xfer += iprot->readListBegin(_etype1448, _size1445);
-            this->new_parts.resize(_size1445);
-            uint32_t _i1449;
-            for (_i1449 = 0; _i1449 < _size1445; ++_i1449)
+            uint32_t _size1469;
+            ::apache::thrift::protocol::TType _etype1472;
+            xfer += iprot->readListBegin(_etype1472, _size1469);
+            this->new_parts.resize(_size1469);
+            uint32_t _i1473;
+            for (_i1473 = 0; _i1473 < _size1469; ++_i1473)
             {
-              xfer += this->new_parts[_i1449].read(iprot);
+              xfer += this->new_parts[_i1473].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12799,10 +12799,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 _iter1450;
-    for (_iter1450 = this->new_parts.begin(); _iter1450 != this->new_parts.end(); ++_iter1450)
+    std::vector<Partition> ::const_iterator _iter1474;
+    for (_iter1474 = this->new_parts.begin(); _iter1474 != this->new_parts.end(); ++_iter1474)
     {
-      xfer += (*_iter1450).write(oprot);
+      xfer += (*_iter1474).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12826,10 +12826,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 _iter1451;
-    for (_iter1451 = (*(this->new_parts)).begin(); _iter1451 != (*(this->new_parts)).end(); ++_iter1451)
+    std::vector<Partition> ::const_iterator _iter1475;
+    for (_iter1475 = (*(this->new_parts)).begin(); _iter1475 != (*(this->new_parts)).end(); ++_iter1475)
     {
-      xfer += (*_iter1451).write(oprot);
+      xfer += (*_iter1475).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13038,14 +13038,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 _size1452;
-            ::apache::thrift::protocol::TType _etype1455;
-            xfer += iprot->readListBegin(_etype1455, _size1452);
-            this->new_parts.resize(_size1452);
-            uint32_t _i1456;
-            for (_i1456 = 0; _i1456 < _size1452; ++_i1456)
+            uint32_t _size1476;
+            ::apache::thrift::protocol::TType _etype1479;
+            xfer += iprot->readListBegin(_etype1479, _size1476);
+            this->new_parts.resize(_size1476);
+            uint32_t _i1480;
+            for (_i1480 = 0; _i1480 < _size1476; ++_i1480)
             {
-              xfer += this->new_parts[_i1456].read(iprot);
+              xfer += this->new_parts[_i1480].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13074,10 +13074,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 _iter1457;
-    for (_iter1457 = this->new_parts.begin(); _iter1457 != this->new_parts.end(); ++_iter1457)
+    std::vector<PartitionSpec> ::const_iterator _iter1481;
+    for (_iter1481 = this->new_parts.begin(); _iter1481 != this->new_parts.end(); ++_iter1481)
     {
-      xfer += (*_iter1457).write(oprot);
+      xfer += (*_iter1481).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13101,10 +13101,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 _iter1458;
-    for (_iter1458 = (*(this->new_parts)).begin(); _iter1458 != (*(this->new_parts)).end(); ++_iter1458)
+    std::vector<PartitionSpec> ::const_iterator _iter1482;
+    for (_iter1482 = (*(this->new_parts)).begin(); _iter1482 != (*(this->new_parts)).end(); ++_iter1482)
     {
-      xfer += (*_iter1458).write(oprot);
+      xfer += (*_iter1482).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13329,14 +13329,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1459;
-            ::apache::thrift::protocol::TType _etype1462;
-            xfer += iprot->readListBegin(_etype1462, _size1459);
-            this->part_vals.resize(_size1459);
-            uint32_t _i1463;
-            for (_i1463 = 0; _i1463 < _size1459; ++_i1463)
+            uint32_t _size1483;
+            ::apache::thrift::protocol::TType _etype1486;
+            xfer += iprot->readListBegin(_etype1486, _size1483);
+            this->part_vals.resize(_size1483);
+            uint32_t _i1487;
+            for (_i1487 = 0; _i1487 < _size1483; ++_i1487)
             {
-              xfer += iprot->readString(this->part_vals[_i1463]);
+              xfer += iprot->readString(this->part_vals[_i1487]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13373,10 +13373,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 _iter1464;
-    for (_iter1464 = this->part_vals.begin(); _iter1464 != this->part_vals.end(); ++_iter1464)
+    std::vector<std::string> ::const_iterator _iter1488;
+    for (_iter1488 = this->part_vals.begin(); _iter1488 != this->part_vals.end(); ++_iter1488)
     {
-      xfer += oprot->writeString((*_iter1464));
+      xfer += oprot->writeString((*_iter1488));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13408,10 +13408,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 _iter1465;
-    for (_iter1465 = (*(this->part_vals)).begin(); _iter1465 != (*(this->part_vals)).end(); ++_iter1465)
+    std::vector<std::string> ::const_iterator _iter1489;
+    for (_iter1489 = (*(this->part_vals)).begin(); _iter1489 != (*(this->part_vals)).end(); ++_iter1489)
     {
-      xfer += oprot->writeString((*_iter1465));
+      xfer += oprot->writeString((*_iter1489));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13883,14 +13883,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1466;
-            ::apache::thrift::protocol::TType _etype1469;
-            xfer += iprot->readListBegin(_etype1469, _size1466);
-            this->part_vals.resize(_size1466);
-            uint32_t _i1470;
-            for (_i1470 = 0; _i1470 < _size1466; ++_i1470)
+            uint32_t _size1490;
+            ::apache::thrift::protocol::TType _etype1493;
+            xfer += iprot->readListBegin(_etype1493, _size1490);
+            this->part_vals.resize(_size1490);
+            uint32_t _i1494;
+            for (_i1494 = 0; _i1494 < _size1490; ++_i1494)
             {
-              xfer += iprot->readString(this->part_vals[_i1470]);
+              xfer += iprot->readString(this->part_vals[_i1494]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13935,10 +13935,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 _iter1471;
-    for (_iter1471 = this->part_vals.begin(); _iter1471 != this->part_vals.end(); ++_iter1471)
+    std::vector<std::string> ::const_iterator _iter1495;
+    for (_iter1495 = this->part_vals.begin(); _iter1495 != this->part_vals.end(); ++_iter1495)
     {
-      xfer += oprot->writeString((*_iter1471));
+      xfer += oprot->writeString((*_iter1495));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13974,10 +13974,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 _iter1472;
-    for (_iter1472 = (*(this->part_vals)).begin(); _iter1472 != (*(this->part_vals)).end(); ++_iter1472)
+    std::vector<std::string> ::const_iterator _iter1496;
+    for (_iter1496 = (*(this->part_vals)).begin(); _iter1496 != (*(this->part_vals)).end(); ++_iter1496)
     {
-      xfer += oprot->writeString((*_iter1472));
+      xfer += oprot->writeString((*_iter1496));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14780,14 +14780,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1473;
-            ::apache::thrift::protocol::TType _etype1476;
-            xfer += iprot->readListBegin(_etype1476, _size1473);
-            this->part_vals.resize(_size1473);
-            uint32_t _i1477;
-            for (_i1477 = 0; _i1477 < _size1473; ++_i1477)
+            uint32_t _size1497;
+            ::apache::thrift::protocol::TType _etype1500;
+            xfer += iprot->readListBegin(_etype1500, _size1497);
+            this->part_vals.resize(_size1497);
+            uint32_t _i1501;
+            for (_i1501 = 0; _i1501 < _size1497; ++_i1501)
             {
-              xfer += iprot->readString(this->part_vals[_i1477]);
+              xfer += iprot->readString(this->part_vals[_i1501]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14832,10 +14832,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 _iter1478;
-    for (_iter1478 = this->part_vals.begin(); _iter1478 != this->part_vals.end(); ++_iter1478)
+    std::vector<std::string> ::const_iterator _iter1502;
+    for (_iter1502 = this->part_vals.begin(); _iter1502 != this->part_vals.end(); ++_iter1502)
     {
-      xfer += oprot->writeString((*_iter1478));
+      xfer += oprot->writeString((*_iter1502));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14871,10 +14871,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 _iter1479;
-    for (_iter1479 = (*(this->part_vals)).begin(); _iter1479 != (*(this->part_vals)).end(); ++_iter1479)
+    std::vector<std::string> ::const_iterator _iter1503;
+    for (_iter1503 = (*(this->part_vals)).begin(); _iter1503 != (*(this->part_vals)).end(); ++_iter1503)
     {
-      xfer += oprot->writeString((*_iter1479));
+      xfer += oprot->writeString((*_iter1503));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15083,14 +15083,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1480;
-            ::apache::thrift::protocol::TType _etype1483;
-            xfer += iprot->readListBegin(_etype1483, _size1480);
-            this->part_vals.resize(_size1480);
-            uint32_t _i1484;
-            for (_i1484 = 0; _i1484 < _size1480; ++_i1484)
+            uint32_t _size1504;
+            ::apache::thrift::protocol::TType _etype1507;
+            xfer += iprot->readListBegin(_etype1507, _size1504);
+            this->part_vals.resize(_size1504);
+            uint32_t _i1508;
+            for (_i1508 = 0; _i1508 < _size1504; ++_i1508)
             {
-              xfer += iprot->readString(this->part_vals[_i1484]);
+              xfer += iprot->readString(this->part_vals[_i1508]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15143,10 +15143,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 _iter1485;
-    for (_iter1485 = this->part_vals.begin(); _iter1485 != this->part_vals.end(); ++_iter1485)
+    std::vector<std::string> ::const_iterator _iter1509;
+    for (_iter1509 = this->part_vals.begin(); _iter1509 != this->part_vals.end(); ++_iter1509)
     {
-      xfer += oprot->writeString((*_iter1485));
+      xfer += oprot->writeString((*_iter1509));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15186,10 +15186,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 _iter1486;
-    for (_iter1486 = (*(this->part_vals)).begin(); _iter1486 != (*(this->part_vals)).end(); ++_iter1486)
+    std::vector<std::string> ::const_iterator _iter1510;
+    for (_iter1510 = (*(this->part_vals)).begin(); _iter1510 != (*(this->part_vals)).end(); ++_iter1510)
     {
-      xfer += oprot->writeString((*_iter1486));
+      xfer += oprot->writeString((*_iter1510));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16195,14 +16195,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1487;
-            ::apache::thrift::protocol::TType _etype1490;
-            xfer += iprot->readListBegin(_etype1490, _size1487);
-            this->part_vals.resize(_size1487);
-            uint32_t _i1491;
-            for (_i1491 = 0; _i1491 < _size1487; ++_i1491)
+            uint32_t _size1511;
+            ::apache::thrift::protocol::TType _etype1514;
+            xfer += iprot->readListBegin(_etype1514, _size1511);
+            this->part_vals.resize(_size1511);
+            uint32_t _i1515;
+            for (_i1515 = 0; _i1515 < _size1511; ++_i1515)
             {
-              xfer += iprot->readString(this->part_vals[_i1491]);
+              xfer += iprot->readString(this->part_vals[_i1515]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16239,10 +16239,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 _iter1492;
-    for (_iter1492 = this->part_vals.begin(); _iter1492 != this->part_vals.end(); ++_iter1492)
+    std::vector<std::string> ::const_iterator _iter1516;
+    for (_iter1516 = this->part_vals.begin(); _iter1516 != this->part_vals.end(); ++_iter1516)
     {
-      xfer += oprot->writeString((*_iter1492));
+      xfer += oprot->writeString((*_iter1516));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16274,10 +16274,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 _iter1493;
-    for (_iter1493 = (*(this->part_vals)).begin(); _iter1493 != (*(this->part_vals)).end(); ++_iter1493)
+    std::vector<std::string> ::const_iterator _iter1517;
+    for (_iter1517 = (*(this->part_vals)).begin(); _iter1517 != (*(this->part_vals)).end(); ++_iter1517)
     {
-      xfer += oprot->writeString((*_iter1493));
+      xfer += oprot->writeString((*_iter1517));
     }
     xfer += oprot->writeListEnd();
 

<TRUNCATED>

[10/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
index 27b6cf8..af62ca1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -436,13 +436,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FULL_TABLE_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
-                struct.fullTableNames = new ArrayList<String>(_list602.size);
-                String _elem603;
-                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.fullTableNames = new ArrayList<String>(_list610.size);
+                String _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem603 = iprot.readString();
-                  struct.fullTableNames.add(_elem603);
+                  _elem611 = iprot.readString();
+                  struct.fullTableNames.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FULL_TABLE_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fullTableNames.size()));
-          for (String _iter605 : struct.fullTableNames)
+          for (String _iter613 : struct.fullTableNames)
           {
-            oprot.writeString(_iter605);
+            oprot.writeString(_iter613);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fullTableNames.size());
-        for (String _iter606 : struct.fullTableNames)
+        for (String _iter614 : struct.fullTableNames)
         {
-          oprot.writeString(_iter606);
+          oprot.writeString(_iter614);
         }
       }
       oprot.writeString(struct.validTxnList);
@@ -520,13 +520,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.fullTableNames = new ArrayList<String>(_list607.size);
-        String _elem608;
-        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.fullTableNames = new ArrayList<String>(_list615.size);
+        String _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem608 = iprot.readString();
-          struct.fullTableNames.add(_elem608);
+          _elem616 = iprot.readString();
+          struct.fullTableNames.add(_elem616);
         }
       }
       struct.setFullTableNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
index 7a1bbc7..615a422 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TBL_VALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list618.size);
-                TableValidWriteIds _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list626.size);
+                TableValidWriteIds _elem627;
+                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                 {
-                  _elem619 = new TableValidWriteIds();
-                  _elem619.read(iprot);
-                  struct.tblValidWriteIds.add(_elem619);
+                  _elem627 = new TableValidWriteIds();
+                  _elem627.read(iprot);
+                  struct.tblValidWriteIds.add(_elem627);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TBL_VALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tblValidWriteIds.size()));
-          for (TableValidWriteIds _iter621 : struct.tblValidWriteIds)
+          for (TableValidWriteIds _iter629 : struct.tblValidWriteIds)
           {
-            _iter621.write(oprot);
+            _iter629.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tblValidWriteIds.size());
-        for (TableValidWriteIds _iter622 : struct.tblValidWriteIds)
+        for (TableValidWriteIds _iter630 : struct.tblValidWriteIds)
         {
-          _iter622.write(oprot);
+          _iter630.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list623.size);
-        TableValidWriteIds _elem624;
-        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list631.size);
+        TableValidWriteIds _elem632;
+        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
         {
-          _elem624 = new TableValidWriteIds();
-          _elem624.read(iprot);
-          struct.tblValidWriteIds.add(_elem624);
+          _elem632 = new TableValidWriteIds();
+          _elem632.read(iprot);
+          struct.tblValidWriteIds.add(_elem632);
         }
       }
       struct.setTblValidWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 4999215..a3dceab 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set666 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set666.size);
-                long _elem667;
-                for (int _i668 = 0; _i668 < _set666.size; ++_i668)
+                org.apache.thrift.protocol.TSet _set674 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set674.size);
+                long _elem675;
+                for (int _i676 = 0; _i676 < _set674.size; ++_i676)
                 {
-                  _elem667 = iprot.readI64();
-                  struct.aborted.add(_elem667);
+                  _elem675 = iprot.readI64();
+                  struct.aborted.add(_elem675);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set669 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set669.size);
-                long _elem670;
-                for (int _i671 = 0; _i671 < _set669.size; ++_i671)
+                org.apache.thrift.protocol.TSet _set677 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set677.size);
+                long _elem678;
+                for (int _i679 = 0; _i679 < _set677.size; ++_i679)
                 {
-                  _elem670 = iprot.readI64();
-                  struct.nosuch.add(_elem670);
+                  _elem678 = iprot.readI64();
+                  struct.nosuch.add(_elem678);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter672 : struct.aborted)
+          for (long _iter680 : struct.aborted)
           {
-            oprot.writeI64(_iter672);
+            oprot.writeI64(_iter680);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter673 : struct.nosuch)
+          for (long _iter681 : struct.nosuch)
           {
-            oprot.writeI64(_iter673);
+            oprot.writeI64(_iter681);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter674 : struct.aborted)
+        for (long _iter682 : struct.aborted)
         {
-          oprot.writeI64(_iter674);
+          oprot.writeI64(_iter682);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter675 : struct.nosuch)
+        for (long _iter683 : struct.nosuch)
         {
-          oprot.writeI64(_iter675);
+          oprot.writeI64(_iter683);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set676 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set676.size);
-        long _elem677;
-        for (int _i678 = 0; _i678 < _set676.size; ++_i678)
+        org.apache.thrift.protocol.TSet _set684 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set684.size);
+        long _elem685;
+        for (int _i686 = 0; _i686 < _set684.size; ++_i686)
         {
-          _elem677 = iprot.readI64();
-          struct.aborted.add(_elem677);
+          _elem685 = iprot.readI64();
+          struct.aborted.add(_elem685);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set679 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set679.size);
-        long _elem680;
-        for (int _i681 = 0; _i681 < _set679.size; ++_i681)
+        org.apache.thrift.protocol.TSet _set687 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set687.size);
+        long _elem688;
+        for (int _i689 = 0; _i689 < _set687.size; ++_i689)
         {
-          _elem680 = iprot.readI64();
-          struct.nosuch.add(_elem680);
+          _elem688 = iprot.readI64();
+          struct.nosuch.add(_elem688);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 0a240e0..4a9824b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField REPLACE_FIELD_DESC = new org.apache.thrift.protocol.TField("replace", org.apache.thrift.protocol.TType.BOOL, (short)1);
   private static final org.apache.thrift.protocol.TField FILES_ADDED_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAdded", org.apache.thrift.protocol.TType.LIST, (short)2);
   private static final org.apache.thrift.protocol.TField FILES_ADDED_CHECKSUM_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAddedChecksum", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField SUB_DIRECTORY_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("subDirectoryList", org.apache.thrift.protocol.TType.LIST, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private boolean replace; // optional
   private List<String> filesAdded; // required
   private List<String> filesAddedChecksum; // optional
+  private List<String> subDirectoryList; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     REPLACE((short)1, "replace"),
     FILES_ADDED((short)2, "filesAdded"),
-    FILES_ADDED_CHECKSUM((short)3, "filesAddedChecksum");
+    FILES_ADDED_CHECKSUM((short)3, "filesAddedChecksum"),
+    SUB_DIRECTORY_LIST((short)4, "subDirectoryList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return FILES_ADDED;
         case 3: // FILES_ADDED_CHECKSUM
           return FILES_ADDED_CHECKSUM;
+        case 4: // SUB_DIRECTORY_LIST
+          return SUB_DIRECTORY_LIST;
         default:
           return null;
       }
@@ -119,7 +124,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __REPLACE_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.REPLACE,_Fields.FILES_ADDED_CHECKSUM};
+  private static final _Fields optionals[] = {_Fields.REPLACE,_Fields.FILES_ADDED_CHECKSUM,_Fields.SUB_DIRECTORY_LIST};
   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);
@@ -131,6 +136,9 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.FILES_ADDED_CHECKSUM, new org.apache.thrift.meta_data.FieldMetaData("filesAddedChecksum", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.SUB_DIRECTORY_LIST, new org.apache.thrift.meta_data.FieldMetaData("subDirectoryList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(InsertEventRequestData.class, metaDataMap);
   }
@@ -159,6 +167,10 @@ import org.slf4j.LoggerFactory;
       List<String> __this__filesAddedChecksum = new ArrayList<String>(other.filesAddedChecksum);
       this.filesAddedChecksum = __this__filesAddedChecksum;
     }
+    if (other.isSetSubDirectoryList()) {
+      List<String> __this__subDirectoryList = new ArrayList<String>(other.subDirectoryList);
+      this.subDirectoryList = __this__subDirectoryList;
+    }
   }
 
   public InsertEventRequestData deepCopy() {
@@ -171,6 +183,7 @@ import org.slf4j.LoggerFactory;
     this.replace = false;
     this.filesAdded = null;
     this.filesAddedChecksum = null;
+    this.subDirectoryList = null;
   }
 
   public boolean isReplace() {
@@ -271,6 +284,44 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getSubDirectoryListSize() {
+    return (this.subDirectoryList == null) ? 0 : this.subDirectoryList.size();
+  }
+
+  public java.util.Iterator<String> getSubDirectoryListIterator() {
+    return (this.subDirectoryList == null) ? null : this.subDirectoryList.iterator();
+  }
+
+  public void addToSubDirectoryList(String elem) {
+    if (this.subDirectoryList == null) {
+      this.subDirectoryList = new ArrayList<String>();
+    }
+    this.subDirectoryList.add(elem);
+  }
+
+  public List<String> getSubDirectoryList() {
+    return this.subDirectoryList;
+  }
+
+  public void setSubDirectoryList(List<String> subDirectoryList) {
+    this.subDirectoryList = subDirectoryList;
+  }
+
+  public void unsetSubDirectoryList() {
+    this.subDirectoryList = null;
+  }
+
+  /** Returns true if field subDirectoryList is set (has been assigned a value) and false otherwise */
+  public boolean isSetSubDirectoryList() {
+    return this.subDirectoryList != null;
+  }
+
+  public void setSubDirectoryListIsSet(boolean value) {
+    if (!value) {
+      this.subDirectoryList = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case REPLACE:
@@ -297,6 +348,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case SUB_DIRECTORY_LIST:
+      if (value == null) {
+        unsetSubDirectoryList();
+      } else {
+        setSubDirectoryList((List<String>)value);
+      }
+      break;
+
     }
   }
 
@@ -311,6 +370,9 @@ import org.slf4j.LoggerFactory;
     case FILES_ADDED_CHECKSUM:
       return getFilesAddedChecksum();
 
+    case SUB_DIRECTORY_LIST:
+      return getSubDirectoryList();
+
     }
     throw new IllegalStateException();
   }
@@ -328,6 +390,8 @@ import org.slf4j.LoggerFactory;
       return isSetFilesAdded();
     case FILES_ADDED_CHECKSUM:
       return isSetFilesAddedChecksum();
+    case SUB_DIRECTORY_LIST:
+      return isSetSubDirectoryList();
     }
     throw new IllegalStateException();
   }
@@ -372,6 +436,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_subDirectoryList = true && this.isSetSubDirectoryList();
+    boolean that_present_subDirectoryList = true && that.isSetSubDirectoryList();
+    if (this_present_subDirectoryList || that_present_subDirectoryList) {
+      if (!(this_present_subDirectoryList && that_present_subDirectoryList))
+        return false;
+      if (!this.subDirectoryList.equals(that.subDirectoryList))
+        return false;
+    }
+
     return true;
   }
 
@@ -394,6 +467,11 @@ import org.slf4j.LoggerFactory;
     if (present_filesAddedChecksum)
       list.add(filesAddedChecksum);
 
+    boolean present_subDirectoryList = true && (isSetSubDirectoryList());
+    list.add(present_subDirectoryList);
+    if (present_subDirectoryList)
+      list.add(subDirectoryList);
+
     return list.hashCode();
   }
 
@@ -435,6 +513,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetSubDirectoryList()).compareTo(other.isSetSubDirectoryList());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSubDirectoryList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.subDirectoryList, other.subDirectoryList);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -478,6 +566,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetSubDirectoryList()) {
+      if (!first) sb.append(", ");
+      sb.append("subDirectoryList:");
+      if (this.subDirectoryList == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.subDirectoryList);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -538,13 +636,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list724.size);
-                String _elem725;
-                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list732.size);
+                String _elem733;
+                for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                 {
-                  _elem725 = iprot.readString();
-                  struct.filesAdded.add(_elem725);
+                  _elem733 = iprot.readString();
+                  struct.filesAdded.add(_elem733);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +654,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list727 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list727.size);
-                String _elem728;
-                for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+                org.apache.thrift.protocol.TList _list735 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list735.size);
+                String _elem736;
+                for (int _i737 = 0; _i737 < _list735.size; ++_i737)
                 {
-                  _elem728 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem728);
+                  _elem736 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem736);
                 }
                 iprot.readListEnd();
               }
@@ -571,6 +669,24 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // SUB_DIRECTORY_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
+                struct.subDirectoryList = new ArrayList<String>(_list738.size);
+                String _elem739;
+                for (int _i740 = 0; _i740 < _list738.size; ++_i740)
+                {
+                  _elem739 = iprot.readString();
+                  struct.subDirectoryList.add(_elem739);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSubDirectoryListIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -593,9 +709,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter730 : struct.filesAdded)
+          for (String _iter741 : struct.filesAdded)
           {
-            oprot.writeString(_iter730);
+            oprot.writeString(_iter741);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +722,23 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter731 : struct.filesAddedChecksum)
+            for (String _iter742 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter731);
+              oprot.writeString(_iter742);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.subDirectoryList != null) {
+        if (struct.isSetSubDirectoryList()) {
+          oprot.writeFieldBegin(SUB_DIRECTORY_LIST_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.subDirectoryList.size()));
+            for (String _iter743 : struct.subDirectoryList)
+            {
+              oprot.writeString(_iter743);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +764,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter732 : struct.filesAdded)
+        for (String _iter744 : struct.filesAdded)
         {
-          oprot.writeString(_iter732);
+          oprot.writeString(_iter744);
         }
       }
       BitSet optionals = new BitSet();
@@ -646,16 +776,28 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSubDirectoryList()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetReplace()) {
         oprot.writeBool(struct.replace);
       }
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter733 : struct.filesAddedChecksum)
+          for (String _iter745 : struct.filesAddedChecksum)
+          {
+            oprot.writeString(_iter745);
+          }
+        }
+      }
+      if (struct.isSetSubDirectoryList()) {
+        {
+          oprot.writeI32(struct.subDirectoryList.size());
+          for (String _iter746 : struct.subDirectoryList)
           {
-            oprot.writeString(_iter733);
+            oprot.writeString(_iter746);
           }
         }
       }
@@ -665,34 +807,47 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list734 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list734.size);
-        String _elem735;
-        for (int _i736 = 0; _i736 < _list734.size; ++_i736)
+        org.apache.thrift.protocol.TList _list747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list747.size);
+        String _elem748;
+        for (int _i749 = 0; _i749 < _list747.size; ++_i749)
         {
-          _elem735 = iprot.readString();
-          struct.filesAdded.add(_elem735);
+          _elem748 = iprot.readString();
+          struct.filesAdded.add(_elem748);
         }
       }
       struct.setFilesAddedIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.replace = iprot.readBool();
         struct.setReplaceIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list737.size);
-          String _elem738;
-          for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+          org.apache.thrift.protocol.TList _list750 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list750.size);
+          String _elem751;
+          for (int _i752 = 0; _i752 < _list750.size; ++_i752)
           {
-            _elem738 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem738);
+            _elem751 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem751);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);
       }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.subDirectoryList = new ArrayList<String>(_list753.size);
+          String _elem754;
+          for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+          {
+            _elem754 = iprot.readString();
+            struct.subDirectoryList.add(_elem754);
+          }
+        }
+        struct.setSubDirectoryListIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index d0dc21c..d4eed32 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list650.size);
-                LockComponent _elem651;
-                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list658.size);
+                LockComponent _elem659;
+                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                 {
-                  _elem651 = new LockComponent();
-                  _elem651.read(iprot);
-                  struct.component.add(_elem651);
+                  _elem659 = new LockComponent();
+                  _elem659.read(iprot);
+                  struct.component.add(_elem659);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter653 : struct.component)
+          for (LockComponent _iter661 : struct.component)
           {
-            _iter653.write(oprot);
+            _iter661.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter654 : struct.component)
+        for (LockComponent _iter662 : struct.component)
         {
-          _iter654.write(oprot);
+          _iter662.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list655.size);
-        LockComponent _elem656;
-        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+        org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list663.size);
+        LockComponent _elem664;
+        for (int _i665 = 0; _i665 < _list663.size; ++_i665)
         {
-          _elem656 = new LockComponent();
-          _elem656.read(iprot);
-          struct.component.add(_elem656);
+          _elem664 = new LockComponent();
+          _elem664.read(iprot);
+          struct.component.add(_elem664);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index 4e792bc..3510995 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -589,13 +589,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set840 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set840.size);
-                String _elem841;
-                for (int _i842 = 0; _i842 < _set840.size; ++_i842)
+                org.apache.thrift.protocol.TSet _set864 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set864.size);
+                String _elem865;
+                for (int _i866 = 0; _i866 < _set864.size; ++_i866)
                 {
-                  _elem841 = iprot.readString();
-                  struct.tablesUsed.add(_elem841);
+                  _elem865 = iprot.readString();
+                  struct.tablesUsed.add(_elem865);
                 }
                 iprot.readSetEnd();
               }
@@ -645,9 +645,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter843 : struct.tablesUsed)
+          for (String _iter867 : struct.tablesUsed)
           {
-            oprot.writeString(_iter843);
+            oprot.writeString(_iter867);
           }
           oprot.writeSetEnd();
         }
@@ -689,9 +689,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter844 : struct.tablesUsed)
+        for (String _iter868 : struct.tablesUsed)
         {
-          oprot.writeString(_iter844);
+          oprot.writeString(_iter868);
         }
       }
       BitSet optionals = new BitSet();
@@ -720,13 +720,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set845 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set845.size);
-        String _elem846;
-        for (int _i847 = 0; _i847 < _set845.size; ++_i847)
+        org.apache.thrift.protocol.TSet _set869 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set869.size);
+        String _elem870;
+        for (int _i871 = 0; _i871 < _set869.size; ++_i871)
         {
-          _elem846 = iprot.readString();
-          struct.tablesUsed.add(_elem846);
+          _elem870 = iprot.readString();
+          struct.tablesUsed.add(_elem870);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index 0c850fa..9228c39 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list716.size);
-                NotificationEvent _elem717;
-                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list724.size);
+                NotificationEvent _elem725;
+                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                 {
-                  _elem717 = new NotificationEvent();
-                  _elem717.read(iprot);
-                  struct.events.add(_elem717);
+                  _elem725 = new NotificationEvent();
+                  _elem725.read(iprot);
+                  struct.events.add(_elem725);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter719 : struct.events)
+          for (NotificationEvent _iter727 : struct.events)
           {
-            _iter719.write(oprot);
+            _iter727.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter720 : struct.events)
+        for (NotificationEvent _iter728 : struct.events)
         {
-          _iter720.write(oprot);
+          _iter728.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list721.size);
-        NotificationEvent _elem722;
-        for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+        org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list729.size);
+        NotificationEvent _elem730;
+        for (int _i731 = 0; _i731 < _list729.size; ++_i731)
         {
-          _elem722 = new NotificationEvent();
-          _elem722.read(iprot);
-          struct.events.add(_elem722);
+          _elem730 = new NotificationEvent();
+          _elem730.read(iprot);
+          struct.events.add(_elem730);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 77c260d..7d9ebba 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list784 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list784.size);
-                long _elem785;
-                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list808.size);
+                long _elem809;
+                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
                 {
-                  _elem785 = iprot.readI64();
-                  struct.fileIds.add(_elem785);
+                  _elem809 = iprot.readI64();
+                  struct.fileIds.add(_elem809);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list787 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list787.size);
-                ByteBuffer _elem788;
-                for (int _i789 = 0; _i789 < _list787.size; ++_i789)
+                org.apache.thrift.protocol.TList _list811 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list811.size);
+                ByteBuffer _elem812;
+                for (int _i813 = 0; _i813 < _list811.size; ++_i813)
                 {
-                  _elem788 = iprot.readBinary();
-                  struct.metadata.add(_elem788);
+                  _elem812 = iprot.readBinary();
+                  struct.metadata.add(_elem812);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter790 : struct.fileIds)
+          for (long _iter814 : struct.fileIds)
           {
-            oprot.writeI64(_iter790);
+            oprot.writeI64(_iter814);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter791 : struct.metadata)
+          for (ByteBuffer _iter815 : struct.metadata)
           {
-            oprot.writeBinary(_iter791);
+            oprot.writeBinary(_iter815);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter792 : struct.fileIds)
+        for (long _iter816 : struct.fileIds)
         {
-          oprot.writeI64(_iter792);
+          oprot.writeI64(_iter816);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter793 : struct.metadata)
+        for (ByteBuffer _iter817 : struct.metadata)
         {
-          oprot.writeBinary(_iter793);
+          oprot.writeBinary(_iter817);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list794 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list794.size);
-        long _elem795;
-        for (int _i796 = 0; _i796 < _list794.size; ++_i796)
+        org.apache.thrift.protocol.TList _list818 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list818.size);
+        long _elem819;
+        for (int _i820 = 0; _i820 < _list818.size; ++_i820)
         {
-          _elem795 = iprot.readI64();
-          struct.fileIds.add(_elem795);
+          _elem819 = iprot.readI64();
+          struct.fileIds.add(_elem819);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list797.size);
-        ByteBuffer _elem798;
-        for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+        org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list821.size);
+        ByteBuffer _elem822;
+        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
         {
-          _elem798 = iprot.readBinary();
-          struct.metadata.add(_elem798);
+          _elem822 = iprot.readBinary();
+          struct.metadata.add(_elem822);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
index 97bb8a4..0aeca14 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
@@ -813,13 +813,13 @@ import org.slf4j.LoggerFactory;
           case 6: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list594.size);
-                String _elem595;
-                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list602.size);
+                String _elem603;
+                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
                 {
-                  _elem595 = iprot.readString();
-                  struct.partNames.add(_elem595);
+                  _elem603 = iprot.readString();
+                  struct.partNames.add(_elem603);
                 }
                 iprot.readListEnd();
               }
@@ -871,9 +871,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter597 : struct.partNames)
+            for (String _iter605 : struct.partNames)
             {
-              oprot.writeString(_iter597);
+              oprot.writeString(_iter605);
             }
             oprot.writeListEnd();
           }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartNames()) {
         {
           oprot.writeI32(struct.partNames.size());
-          for (String _iter598 : struct.partNames)
+          for (String _iter606 : struct.partNames)
           {
-            oprot.writeString(_iter598);
+            oprot.writeString(_iter606);
           }
         }
       }
@@ -934,13 +934,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partNames = new ArrayList<String>(_list599.size);
-          String _elem600;
-          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+          org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partNames = new ArrayList<String>(_list607.size);
+          String _elem608;
+          for (int _i609 = 0; _i609 < _list607.size; ++_i609)
           {
-            _elem600 = iprot.readString();
-            struct.partNames.add(_elem600);
+            _elem608 = iprot.readString();
+            struct.partNames.add(_elem608);
           }
         }
         struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index 76dfe17..88d7e3f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list912.size);
-                FieldSchema _elem913;
-                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
+                org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list936.size);
+                FieldSchema _elem937;
+                for (int _i938 = 0; _i938 < _list936.size; ++_i938)
                 {
-                  _elem913 = new FieldSchema();
-                  _elem913.read(iprot);
-                  struct.cols.add(_elem913);
+                  _elem937 = new FieldSchema();
+                  _elem937.read(iprot);
+                  struct.cols.add(_elem937);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter915 : struct.cols)
+          for (FieldSchema _iter939 : struct.cols)
           {
-            _iter915.write(oprot);
+            _iter939.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter916 : struct.cols)
+          for (FieldSchema _iter940 : struct.cols)
           {
-            _iter916.write(oprot);
+            _iter940.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list917.size);
-          FieldSchema _elem918;
-          for (int _i919 = 0; _i919 < _list917.size; ++_i919)
+          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list941.size);
+          FieldSchema _elem942;
+          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
           {
-            _elem918 = new FieldSchema();
-            _elem918.read(iprot);
-            struct.cols.add(_elem918);
+            _elem942 = new FieldSchema();
+            _elem942.read(iprot);
+            struct.cols.add(_elem942);
           }
         }
         struct.setColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index d7e5132..9fb037f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list692.size);
-                ShowCompactResponseElement _elem693;
-                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
+                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list700.size);
+                ShowCompactResponseElement _elem701;
+                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                 {
-                  _elem693 = new ShowCompactResponseElement();
-                  _elem693.read(iprot);
-                  struct.compacts.add(_elem693);
+                  _elem701 = new ShowCompactResponseElement();
+                  _elem701.read(iprot);
+                  struct.compacts.add(_elem701);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter695 : struct.compacts)
+          for (ShowCompactResponseElement _iter703 : struct.compacts)
           {
-            _iter695.write(oprot);
+            _iter703.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter696 : struct.compacts)
+        for (ShowCompactResponseElement _iter704 : struct.compacts)
         {
-          _iter696.write(oprot);
+          _iter704.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list697.size);
-        ShowCompactResponseElement _elem698;
-        for (int _i699 = 0; _i699 < _list697.size; ++_i699)
+        org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list705.size);
+        ShowCompactResponseElement _elem706;
+        for (int _i707 = 0; _i707 < _list705.size; ++_i707)
         {
-          _elem698 = new ShowCompactResponseElement();
-          _elem698.read(iprot);
-          struct.compacts.add(_elem698);
+          _elem706 = new ShowCompactResponseElement();
+          _elem706.read(iprot);
+          struct.compacts.add(_elem706);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 0e1009c..e0db2f7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list658.size);
-                ShowLocksResponseElement _elem659;
-                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list666.size);
+                ShowLocksResponseElement _elem667;
+                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                 {
-                  _elem659 = new ShowLocksResponseElement();
-                  _elem659.read(iprot);
-                  struct.locks.add(_elem659);
+                  _elem667 = new ShowLocksResponseElement();
+                  _elem667.read(iprot);
+                  struct.locks.add(_elem667);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter661 : struct.locks)
+          for (ShowLocksResponseElement _iter669 : struct.locks)
           {
-            _iter661.write(oprot);
+            _iter669.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter662 : struct.locks)
+          for (ShowLocksResponseElement _iter670 : struct.locks)
           {
-            _iter662.write(oprot);
+            _iter670.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list663.size);
-          ShowLocksResponseElement _elem664;
-          for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+          org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list671.size);
+          ShowLocksResponseElement _elem672;
+          for (int _i673 = 0; _i673 < _list671.size; ++_i673)
           {
-            _elem664 = new ShowLocksResponseElement();
-            _elem664.read(iprot);
-            struct.locks.add(_elem664);
+            _elem672 = new ShowLocksResponseElement();
+            _elem672.read(iprot);
+            struct.locks.add(_elem672);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
index 20f225d..de15fc6 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
@@ -708,13 +708,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INVALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.invalidWriteIds = new ArrayList<Long>(_list610.size);
-                long _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
+                struct.invalidWriteIds = new ArrayList<Long>(_list618.size);
+                long _elem619;
+                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
                 {
-                  _elem611 = iprot.readI64();
-                  struct.invalidWriteIds.add(_elem611);
+                  _elem619 = iprot.readI64();
+                  struct.invalidWriteIds.add(_elem619);
                 }
                 iprot.readListEnd();
               }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INVALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalidWriteIds.size()));
-          for (long _iter613 : struct.invalidWriteIds)
+          for (long _iter621 : struct.invalidWriteIds)
           {
-            oprot.writeI64(_iter613);
+            oprot.writeI64(_iter621);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.writeIdHighWaterMark);
       {
         oprot.writeI32(struct.invalidWriteIds.size());
-        for (long _iter614 : struct.invalidWriteIds)
+        for (long _iter622 : struct.invalidWriteIds)
         {
-          oprot.writeI64(_iter614);
+          oprot.writeI64(_iter622);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -827,13 +827,13 @@ import org.slf4j.LoggerFactory;
       struct.writeIdHighWaterMark = iprot.readI64();
       struct.setWriteIdHighWaterMarkIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.invalidWriteIds = new ArrayList<Long>(_list615.size);
-        long _elem616;
-        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.invalidWriteIds = new ArrayList<Long>(_list623.size);
+        long _elem624;
+        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
         {
-          _elem616 = iprot.readI64();
-          struct.invalidWriteIds.add(_elem616);
+          _elem624 = iprot.readI64();
+          struct.invalidWriteIds.add(_elem624);
         }
       }
       struct.setInvalidWriteIdsIsSet(true);


[04/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 5402372..add9197 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -189,6 +189,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst)')
   print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
   print('  void flushCache()')
+  print('  WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)')
   print('  CmRecycleResponse cm_recycle(CmRecycleRequest request)')
   print('  GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)')
   print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
@@ -1290,6 +1291,12 @@ elif cmd == 'flushCache':
     sys.exit(1)
   pp.pprint(client.flushCache())
 
+elif cmd == 'add_write_notification_log':
+  if len(args) != 1:
+    print('add_write_notification_log requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_write_notification_log(eval(args[0]),))
+
 elif cmd == 'cm_recycle':
   if len(args) != 1:
     print('cm_recycle requires 1 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 5a3f2c1..a5bcc10 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1309,6 +1309,13 @@ class Iface(fb303.FacebookService.Iface):
   def flushCache(self):
     pass
 
+  def add_write_notification_log(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
   def cm_recycle(self, request):
     """
     Parameters:
@@ -7546,6 +7553,37 @@ class Client(fb303.FacebookService.Client, Iface):
     iprot.readMessageEnd()
     return
 
+  def add_write_notification_log(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_add_write_notification_log(rqst)
+    return self.recv_add_write_notification_log()
+
+  def send_add_write_notification_log(self, rqst):
+    self._oprot.writeMessageBegin('add_write_notification_log', TMessageType.CALL, self._seqid)
+    args = add_write_notification_log_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_write_notification_log(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = add_write_notification_log_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "add_write_notification_log failed: unknown result")
+
   def cm_recycle(self, request):
     """
     Parameters:
@@ -9170,6 +9208,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_notification_events_count"] = Processor.process_get_notification_events_count
     self._processMap["fire_listener_event"] = Processor.process_fire_listener_event
     self._processMap["flushCache"] = Processor.process_flushCache
+    self._processMap["add_write_notification_log"] = Processor.process_add_write_notification_log
     self._processMap["cm_recycle"] = Processor.process_cm_recycle
     self._processMap["get_file_metadata_by_expr"] = Processor.process_get_file_metadata_by_expr
     self._processMap["get_file_metadata"] = Processor.process_get_file_metadata
@@ -13287,6 +13326,25 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_add_write_notification_log(self, seqid, iprot, oprot):
+    args = add_write_notification_log_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_write_notification_log_result()
+    try:
+      result.success = self._handler.add_write_notification_log(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("add_write_notification_log", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_cm_recycle(self, seqid, iprot, oprot):
     args = cm_recycle_args()
     args.read(iprot)
@@ -15987,10 +16045,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype819, _size816) = iprot.readListBegin()
-          for _i820 in xrange(_size816):
-            _elem821 = iprot.readString()
-            self.success.append(_elem821)
+          (_etype840, _size837) = iprot.readListBegin()
+          for _i841 in xrange(_size837):
+            _elem842 = iprot.readString()
+            self.success.append(_elem842)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16013,8 +16071,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 iter822 in self.success:
-        oprot.writeString(iter822)
+      for iter843 in self.success:
+        oprot.writeString(iter843)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16119,10 +16177,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = iprot.readString()
-            self.success.append(_elem828)
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = iprot.readString()
+            self.success.append(_elem849)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16145,8 +16203,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 iter829 in self.success:
-        oprot.writeString(iter829)
+      for iter850 in self.success:
+        oprot.writeString(iter850)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16916,12 +16974,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype831, _vtype832, _size830 ) = iprot.readMapBegin()
-          for _i834 in xrange(_size830):
-            _key835 = iprot.readString()
-            _val836 = Type()
-            _val836.read(iprot)
-            self.success[_key835] = _val836
+          (_ktype852, _vtype853, _size851 ) = iprot.readMapBegin()
+          for _i855 in xrange(_size851):
+            _key856 = iprot.readString()
+            _val857 = Type()
+            _val857.read(iprot)
+            self.success[_key856] = _val857
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16944,9 +17002,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 kiter837,viter838 in self.success.items():
-        oprot.writeString(kiter837)
-        viter838.write(oprot)
+      for kiter858,viter859 in self.success.items():
+        oprot.writeString(kiter858)
+        viter859.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -17089,11 +17147,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype842, _size839) = iprot.readListBegin()
-          for _i843 in xrange(_size839):
-            _elem844 = FieldSchema()
-            _elem844.read(iprot)
-            self.success.append(_elem844)
+          (_etype863, _size860) = iprot.readListBegin()
+          for _i864 in xrange(_size860):
+            _elem865 = FieldSchema()
+            _elem865.read(iprot)
+            self.success.append(_elem865)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17128,8 +17186,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 iter845 in self.success:
-        iter845.write(oprot)
+      for iter866 in self.success:
+        iter866.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17296,11 +17354,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype849, _size846) = iprot.readListBegin()
-          for _i850 in xrange(_size846):
-            _elem851 = FieldSchema()
-            _elem851.read(iprot)
-            self.success.append(_elem851)
+          (_etype870, _size867) = iprot.readListBegin()
+          for _i871 in xrange(_size867):
+            _elem872 = FieldSchema()
+            _elem872.read(iprot)
+            self.success.append(_elem872)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17335,8 +17393,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 iter852 in self.success:
-        iter852.write(oprot)
+      for iter873 in self.success:
+        iter873.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17489,11 +17547,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype856, _size853) = iprot.readListBegin()
-          for _i857 in xrange(_size853):
-            _elem858 = FieldSchema()
-            _elem858.read(iprot)
-            self.success.append(_elem858)
+          (_etype877, _size874) = iprot.readListBegin()
+          for _i878 in xrange(_size874):
+            _elem879 = FieldSchema()
+            _elem879.read(iprot)
+            self.success.append(_elem879)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17528,8 +17586,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 iter859 in self.success:
-        iter859.write(oprot)
+      for iter880 in self.success:
+        iter880.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17696,11 +17754,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype863, _size860) = iprot.readListBegin()
-          for _i864 in xrange(_size860):
-            _elem865 = FieldSchema()
-            _elem865.read(iprot)
-            self.success.append(_elem865)
+          (_etype884, _size881) = iprot.readListBegin()
+          for _i885 in xrange(_size881):
+            _elem886 = FieldSchema()
+            _elem886.read(iprot)
+            self.success.append(_elem886)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17735,8 +17793,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 iter866 in self.success:
-        iter866.write(oprot)
+      for iter887 in self.success:
+        iter887.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18189,66 +18247,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype870, _size867) = iprot.readListBegin()
-          for _i871 in xrange(_size867):
-            _elem872 = SQLPrimaryKey()
-            _elem872.read(iprot)
-            self.primaryKeys.append(_elem872)
+          (_etype891, _size888) = iprot.readListBegin()
+          for _i892 in xrange(_size888):
+            _elem893 = SQLPrimaryKey()
+            _elem893.read(iprot)
+            self.primaryKeys.append(_elem893)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = SQLForeignKey()
-            _elem878.read(iprot)
-            self.foreignKeys.append(_elem878)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = SQLForeignKey()
+            _elem899.read(iprot)
+            self.foreignKeys.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype882, _size879) = iprot.readListBegin()
-          for _i883 in xrange(_size879):
-            _elem884 = SQLUniqueConstraint()
-            _elem884.read(iprot)
-            self.uniqueConstraints.append(_elem884)
+          (_etype903, _size900) = iprot.readListBegin()
+          for _i904 in xrange(_size900):
+            _elem905 = SQLUniqueConstraint()
+            _elem905.read(iprot)
+            self.uniqueConstraints.append(_elem905)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype888, _size885) = iprot.readListBegin()
-          for _i889 in xrange(_size885):
-            _elem890 = SQLNotNullConstraint()
-            _elem890.read(iprot)
-            self.notNullConstraints.append(_elem890)
+          (_etype909, _size906) = iprot.readListBegin()
+          for _i910 in xrange(_size906):
+            _elem911 = SQLNotNullConstraint()
+            _elem911.read(iprot)
+            self.notNullConstraints.append(_elem911)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype894, _size891) = iprot.readListBegin()
-          for _i895 in xrange(_size891):
-            _elem896 = SQLDefaultConstraint()
-            _elem896.read(iprot)
-            self.defaultConstraints.append(_elem896)
+          (_etype915, _size912) = iprot.readListBegin()
+          for _i916 in xrange(_size912):
+            _elem917 = SQLDefaultConstraint()
+            _elem917.read(iprot)
+            self.defaultConstraints.append(_elem917)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype900, _size897) = iprot.readListBegin()
-          for _i901 in xrange(_size897):
-            _elem902 = SQLCheckConstraint()
-            _elem902.read(iprot)
-            self.checkConstraints.append(_elem902)
+          (_etype921, _size918) = iprot.readListBegin()
+          for _i922 in xrange(_size918):
+            _elem923 = SQLCheckConstraint()
+            _elem923.read(iprot)
+            self.checkConstraints.append(_elem923)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18269,43 +18327,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter903 in self.primaryKeys:
-        iter903.write(oprot)
+      for iter924 in self.primaryKeys:
+        iter924.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter904 in self.foreignKeys:
-        iter904.write(oprot)
+      for iter925 in self.foreignKeys:
+        iter925.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter905 in self.uniqueConstraints:
-        iter905.write(oprot)
+      for iter926 in self.uniqueConstraints:
+        iter926.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter906 in self.notNullConstraints:
-        iter906.write(oprot)
+      for iter927 in self.notNullConstraints:
+        iter927.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter907 in self.defaultConstraints:
-        iter907.write(oprot)
+      for iter928 in self.defaultConstraints:
+        iter928.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter908 in self.checkConstraints:
-        iter908.write(oprot)
+      for iter929 in self.checkConstraints:
+        iter929.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19865,10 +19923,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype912, _size909) = iprot.readListBegin()
-          for _i913 in xrange(_size909):
-            _elem914 = iprot.readString()
-            self.partNames.append(_elem914)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = iprot.readString()
+            self.partNames.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19893,8 +19951,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter915 in self.partNames:
-        oprot.writeString(iter915)
+      for iter936 in self.partNames:
+        oprot.writeString(iter936)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20094,10 +20152,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype919, _size916) = iprot.readListBegin()
-          for _i920 in xrange(_size916):
-            _elem921 = iprot.readString()
-            self.success.append(_elem921)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = iprot.readString()
+            self.success.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20120,8 +20178,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 iter922 in self.success:
-        oprot.writeString(iter922)
+      for iter943 in self.success:
+        oprot.writeString(iter943)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20271,10 +20329,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype926, _size923) = iprot.readListBegin()
-          for _i927 in xrange(_size923):
-            _elem928 = iprot.readString()
-            self.success.append(_elem928)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = iprot.readString()
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20297,8 +20355,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter929 in self.success:
-        oprot.writeString(iter929)
+      for iter950 in self.success:
+        oprot.writeString(iter950)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20422,10 +20480,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = iprot.readString()
-            self.success.append(_elem935)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = iprot.readString()
+            self.success.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20448,8 +20506,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter936 in self.success:
-        oprot.writeString(iter936)
+      for iter957 in self.success:
+        oprot.writeString(iter957)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20522,10 +20580,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = iprot.readString()
-            self.tbl_types.append(_elem942)
+          (_etype961, _size958) = iprot.readListBegin()
+          for _i962 in xrange(_size958):
+            _elem963 = iprot.readString()
+            self.tbl_types.append(_elem963)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20550,8 +20608,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter943 in self.tbl_types:
-        oprot.writeString(iter943)
+      for iter964 in self.tbl_types:
+        oprot.writeString(iter964)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20607,11 +20665,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = TableMeta()
-            _elem949.read(iprot)
-            self.success.append(_elem949)
+          (_etype968, _size965) = iprot.readListBegin()
+          for _i969 in xrange(_size965):
+            _elem970 = TableMeta()
+            _elem970.read(iprot)
+            self.success.append(_elem970)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20634,8 +20692,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter950 in self.success:
-        iter950.write(oprot)
+      for iter971 in self.success:
+        iter971.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20759,10 +20817,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype954, _size951) = iprot.readListBegin()
-          for _i955 in xrange(_size951):
-            _elem956 = iprot.readString()
-            self.success.append(_elem956)
+          (_etype975, _size972) = iprot.readListBegin()
+          for _i976 in xrange(_size972):
+            _elem977 = iprot.readString()
+            self.success.append(_elem977)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20785,8 +20843,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 iter957 in self.success:
-        oprot.writeString(iter957)
+      for iter978 in self.success:
+        oprot.writeString(iter978)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21022,10 +21080,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype961, _size958) = iprot.readListBegin()
-          for _i962 in xrange(_size958):
-            _elem963 = iprot.readString()
-            self.tbl_names.append(_elem963)
+          (_etype982, _size979) = iprot.readListBegin()
+          for _i983 in xrange(_size979):
+            _elem984 = iprot.readString()
+            self.tbl_names.append(_elem984)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21046,8 +21104,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 iter964 in self.tbl_names:
-        oprot.writeString(iter964)
+      for iter985 in self.tbl_names:
+        oprot.writeString(iter985)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21099,11 +21157,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype968, _size965) = iprot.readListBegin()
-          for _i969 in xrange(_size965):
-            _elem970 = Table()
-            _elem970.read(iprot)
-            self.success.append(_elem970)
+          (_etype989, _size986) = iprot.readListBegin()
+          for _i990 in xrange(_size986):
+            _elem991 = Table()
+            _elem991.read(iprot)
+            self.success.append(_elem991)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21120,8 +21178,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 iter971 in self.success:
-        iter971.write(oprot)
+      for iter992 in self.success:
+        iter992.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21513,10 +21571,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype975, _size972) = iprot.readListBegin()
-          for _i976 in xrange(_size972):
-            _elem977 = iprot.readString()
-            self.tbl_names.append(_elem977)
+          (_etype996, _size993) = iprot.readListBegin()
+          for _i997 in xrange(_size993):
+            _elem998 = iprot.readString()
+            self.tbl_names.append(_elem998)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21537,8 +21595,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter978 in self.tbl_names:
-        oprot.writeString(iter978)
+      for iter999 in self.tbl_names:
+        oprot.writeString(iter999)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21599,12 +21657,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype980, _vtype981, _size979 ) = iprot.readMapBegin()
-          for _i983 in xrange(_size979):
-            _key984 = iprot.readString()
-            _val985 = Materialization()
-            _val985.read(iprot)
-            self.success[_key984] = _val985
+          (_ktype1001, _vtype1002, _size1000 ) = iprot.readMapBegin()
+          for _i1004 in xrange(_size1000):
+            _key1005 = iprot.readString()
+            _val1006 = Materialization()
+            _val1006.read(iprot)
+            self.success[_key1005] = _val1006
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21639,9 +21697,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter986,viter987 in self.success.items():
-        oprot.writeString(kiter986)
-        viter987.write(oprot)
+      for kiter1007,viter1008 in self.success.items():
+        oprot.writeString(kiter1007)
+        viter1008.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22006,10 +22064,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype991, _size988) = iprot.readListBegin()
-          for _i992 in xrange(_size988):
-            _elem993 = iprot.readString()
-            self.success.append(_elem993)
+          (_etype1012, _size1009) = iprot.readListBegin()
+          for _i1013 in xrange(_size1009):
+            _elem1014 = iprot.readString()
+            self.success.append(_elem1014)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22044,8 +22102,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 iter994 in self.success:
-        oprot.writeString(iter994)
+      for iter1015 in self.success:
+        oprot.writeString(iter1015)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23015,11 +23073,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype998, _size995) = iprot.readListBegin()
-          for _i999 in xrange(_size995):
-            _elem1000 = Partition()
-            _elem1000.read(iprot)
-            self.new_parts.append(_elem1000)
+          (_etype1019, _size1016) = iprot.readListBegin()
+          for _i1020 in xrange(_size1016):
+            _elem1021 = Partition()
+            _elem1021.read(iprot)
+            self.new_parts.append(_elem1021)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23036,8 +23094,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 iter1001 in self.new_parts:
-        iter1001.write(oprot)
+      for iter1022 in self.new_parts:
+        iter1022.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23195,11 +23253,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1005, _size1002) = iprot.readListBegin()
-          for _i1006 in xrange(_size1002):
-            _elem1007 = PartitionSpec()
-            _elem1007.read(iprot)
-            self.new_parts.append(_elem1007)
+          (_etype1026, _size1023) = iprot.readListBegin()
+          for _i1027 in xrange(_size1023):
+            _elem1028 = PartitionSpec()
+            _elem1028.read(iprot)
+            self.new_parts.append(_elem1028)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23216,8 +23274,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 iter1008 in self.new_parts:
-        iter1008.write(oprot)
+      for iter1029 in self.new_parts:
+        iter1029.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23391,10 +23449,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1012, _size1009) = iprot.readListBegin()
-          for _i1013 in xrange(_size1009):
-            _elem1014 = iprot.readString()
-            self.part_vals.append(_elem1014)
+          (_etype1033, _size1030) = iprot.readListBegin()
+          for _i1034 in xrange(_size1030):
+            _elem1035 = iprot.readString()
+            self.part_vals.append(_elem1035)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23419,8 +23477,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 iter1015 in self.part_vals:
-        oprot.writeString(iter1015)
+      for iter1036 in self.part_vals:
+        oprot.writeString(iter1036)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23773,10 +23831,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1019, _size1016) = iprot.readListBegin()
-          for _i1020 in xrange(_size1016):
-            _elem1021 = iprot.readString()
-            self.part_vals.append(_elem1021)
+          (_etype1040, _size1037) = iprot.readListBegin()
+          for _i1041 in xrange(_size1037):
+            _elem1042 = iprot.readString()
+            self.part_vals.append(_elem1042)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23807,8 +23865,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 iter1022 in self.part_vals:
-        oprot.writeString(iter1022)
+      for iter1043 in self.part_vals:
+        oprot.writeString(iter1043)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -24403,10 +24461,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1026, _size1023) = iprot.readListBegin()
-          for _i1027 in xrange(_size1023):
-            _elem1028 = iprot.readString()
-            self.part_vals.append(_elem1028)
+          (_etype1047, _size1044) = iprot.readListBegin()
+          for _i1048 in xrange(_size1044):
+            _elem1049 = iprot.readString()
+            self.part_vals.append(_elem1049)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24436,8 +24494,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 iter1029 in self.part_vals:
-        oprot.writeString(iter1029)
+      for iter1050 in self.part_vals:
+        oprot.writeString(iter1050)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -24610,10 +24668,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1033, _size1030) = iprot.readListBegin()
-          for _i1034 in xrange(_size1030):
-            _elem1035 = iprot.readString()
-            self.part_vals.append(_elem1035)
+          (_etype1054, _size1051) = iprot.readListBegin()
+          for _i1055 in xrange(_size1051):
+            _elem1056 = iprot.readString()
+            self.part_vals.append(_elem1056)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24649,8 +24707,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 iter1036 in self.part_vals:
-        oprot.writeString(iter1036)
+      for iter1057 in self.part_vals:
+        oprot.writeString(iter1057)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -25387,10 +25445,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1040, _size1037) = iprot.readListBegin()
-          for _i1041 in xrange(_size1037):
-            _elem1042 = iprot.readString()
-            self.part_vals.append(_elem1042)
+          (_etype1061, _size1058) = iprot.readListBegin()
+          for _i1062 in xrange(_size1058):
+            _elem1063 = iprot.readString()
+            self.part_vals.append(_elem1063)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25415,8 +25473,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 iter1043 in self.part_vals:
-        oprot.writeString(iter1043)
+      for iter1064 in self.part_vals:
+        oprot.writeString(iter1064)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25575,11 +25633,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1045, _vtype1046, _size1044 ) = iprot.readMapBegin()
-          for _i1048 in xrange(_size1044):
-            _key1049 = iprot.readString()
-            _val1050 = iprot.readString()
-            self.partitionSpecs[_key1049] = _val1050
+          (_ktype1066, _vtype1067, _size1065 ) = iprot.readMapBegin()
+          for _i1069 in xrange(_size1065):
+            _key1070 = iprot.readString()
+            _val1071 = iprot.readString()
+            self.partitionSpecs[_key1070] = _val1071
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25616,9 +25674,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 kiter1051,viter1052 in self.partitionSpecs.items():
-        oprot.writeString(kiter1051)
-        oprot.writeString(viter1052)
+      for kiter1072,viter1073 in self.partitionSpecs.items():
+        oprot.writeString(kiter1072)
+        oprot.writeString(viter1073)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25823,11 +25881,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1054, _vtype1055, _size1053 ) = iprot.readMapBegin()
-          for _i1057 in xrange(_size1053):
-            _key1058 = iprot.readString()
-            _val1059 = iprot.readString()
-            self.partitionSpecs[_key1058] = _val1059
+          (_ktype1075, _vtype1076, _size1074 ) = iprot.readMapBegin()
+          for _i1078 in xrange(_size1074):
+            _key1079 = iprot.readString()
+            _val1080 = iprot.readString()
+            self.partitionSpecs[_key1079] = _val1080
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25864,9 +25922,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 kiter1060,viter1061 in self.partitionSpecs.items():
-        oprot.writeString(kiter1060)
-        oprot.writeString(viter1061)
+      for kiter1081,viter1082 in self.partitionSpecs.items():
+        oprot.writeString(kiter1081)
+        oprot.writeString(viter1082)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25949,11 +26007,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1065, _size1062) = iprot.readListBegin()
-          for _i1066 in xrange(_size1062):
-            _elem1067 = Partition()
-            _elem1067.read(iprot)
-            self.success.append(_elem1067)
+          (_etype1086, _size1083) = iprot.readListBegin()
+          for _i1087 in xrange(_size1083):
+            _elem1088 = Partition()
+            _elem1088.read(iprot)
+            self.success.append(_elem1088)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25994,8 +26052,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 iter1068 in self.success:
-        iter1068.write(oprot)
+      for iter1089 in self.success:
+        iter1089.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26089,10 +26147,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1072, _size1069) = iprot.readListBegin()
-          for _i1073 in xrange(_size1069):
-            _elem1074 = iprot.readString()
-            self.part_vals.append(_elem1074)
+          (_etype1093, _size1090) = iprot.readListBegin()
+          for _i1094 in xrange(_size1090):
+            _elem1095 = iprot.readString()
+            self.part_vals.append(_elem1095)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26104,10 +26162,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1078, _size1075) = iprot.readListBegin()
-          for _i1079 in xrange(_size1075):
-            _elem1080 = iprot.readString()
-            self.group_names.append(_elem1080)
+          (_etype1099, _size1096) = iprot.readListBegin()
+          for _i1100 in xrange(_size1096):
+            _elem1101 = iprot.readString()
+            self.group_names.append(_elem1101)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26132,8 +26190,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 iter1081 in self.part_vals:
-        oprot.writeString(iter1081)
+      for iter1102 in self.part_vals:
+        oprot.writeString(iter1102)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -26143,8 +26201,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 iter1082 in self.group_names:
-        oprot.writeString(iter1082)
+      for iter1103 in self.group_names:
+        oprot.writeString(iter1103)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26573,11 +26631,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = Partition()
-            _elem1088.read(iprot)
-            self.success.append(_elem1088)
+          (_etype1107, _size1104) = iprot.readListBegin()
+          for _i1108 in xrange(_size1104):
+            _elem1109 = Partition()
+            _elem1109.read(iprot)
+            self.success.append(_elem1109)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26606,8 +26664,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 iter1089 in self.success:
-        iter1089.write(oprot)
+      for iter1110 in self.success:
+        iter1110.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26701,10 +26759,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1093, _size1090) = iprot.readListBegin()
-          for _i1094 in xrange(_size1090):
-            _elem1095 = iprot.readString()
-            self.group_names.append(_elem1095)
+          (_etype1114, _size1111) = iprot.readListBegin()
+          for _i1115 in xrange(_size1111):
+            _elem1116 = iprot.readString()
+            self.group_names.append(_elem1116)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26737,8 +26795,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 iter1096 in self.group_names:
-        oprot.writeString(iter1096)
+      for iter1117 in self.group_names:
+        oprot.writeString(iter1117)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26799,11 +26857,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1100, _size1097) = iprot.readListBegin()
-          for _i1101 in xrange(_size1097):
-            _elem1102 = Partition()
-            _elem1102.read(iprot)
-            self.success.append(_elem1102)
+          (_etype1121, _size1118) = iprot.readListBegin()
+          for _i1122 in xrange(_size1118):
+            _elem1123 = Partition()
+            _elem1123.read(iprot)
+            self.success.append(_elem1123)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26832,8 +26890,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 iter1103 in self.success:
-        iter1103.write(oprot)
+      for iter1124 in self.success:
+        iter1124.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26991,11 +27049,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1107, _size1104) = iprot.readListBegin()
-          for _i1108 in xrange(_size1104):
-            _elem1109 = PartitionSpec()
-            _elem1109.read(iprot)
-            self.success.append(_elem1109)
+          (_etype1128, _size1125) = iprot.readListBegin()
+          for _i1129 in xrange(_size1125):
+            _elem1130 = PartitionSpec()
+            _elem1130.read(iprot)
+            self.success.append(_elem1130)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27024,8 +27082,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 iter1110 in self.success:
-        iter1110.write(oprot)
+      for iter1131 in self.success:
+        iter1131.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27183,10 +27241,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1114, _size1111) = iprot.readListBegin()
-          for _i1115 in xrange(_size1111):
-            _elem1116 = iprot.readString()
-            self.success.append(_elem1116)
+          (_etype1135, _size1132) = iprot.readListBegin()
+          for _i1136 in xrange(_size1132):
+            _elem1137 = iprot.readString()
+            self.success.append(_elem1137)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27215,8 +27273,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 iter1117 in self.success:
-        oprot.writeString(iter1117)
+      for iter1138 in self.success:
+        oprot.writeString(iter1138)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27456,10 +27514,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1121, _size1118) = iprot.readListBegin()
-          for _i1122 in xrange(_size1118):
-            _elem1123 = iprot.readString()
-            self.part_vals.append(_elem1123)
+          (_etype1142, _size1139) = iprot.readListBegin()
+          for _i1143 in xrange(_size1139):
+            _elem1144 = iprot.readString()
+            self.part_vals.append(_elem1144)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27489,8 +27547,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 iter1124 in self.part_vals:
-        oprot.writeString(iter1124)
+      for iter1145 in self.part_vals:
+        oprot.writeString(iter1145)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27554,11 +27612,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1128, _size1125) = iprot.readListBegin()
-          for _i1129 in xrange(_size1125):
-            _elem1130 = Partition()
-            _elem1130.read(iprot)
-            self.success.append(_elem1130)
+          (_etype1149, _size1146) = iprot.readListBegin()
+          for _i1150 in xrange(_size1146):
+            _elem1151 = Partition()
+            _elem1151.read(iprot)
+            self.success.append(_elem1151)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27587,8 +27645,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 iter1131 in self.success:
-        iter1131.write(oprot)
+      for iter1152 in self.success:
+        iter1152.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27675,10 +27733,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1135, _size1132) = iprot.readListBegin()
-          for _i1136 in xrange(_size1132):
-            _elem1137 = iprot.readString()
-            self.part_vals.append(_elem1137)
+          (_etype1156, _size1153) = iprot.readListBegin()
+          for _i1157 in xrange(_size1153):
+            _elem1158 = iprot.readString()
+            self.part_vals.append(_elem1158)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27695,10 +27753,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1141, _size1138) = iprot.readListBegin()
-          for _i1142 in xrange(_size1138):
-            _elem1143 = iprot.readString()
-            self.group_names.append(_elem1143)
+          (_etype1162, _size1159) = iprot.readListBegin()
+          for _i1163 in xrange(_size1159):
+            _elem1164 = iprot.readString()
+            self.group_names.append(_elem1164)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27723,8 +27781,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 iter1144 in self.part_vals:
-        oprot.writeString(iter1144)
+      for iter1165 in self.part_vals:
+        oprot.writeString(iter1165)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27738,8 +27796,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 iter1145 in self.group_names:
-        oprot.writeString(iter1145)
+      for iter1166 in self.group_names:
+        oprot.writeString(iter1166)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27801,11 +27859,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1149, _size1146) = iprot.readListBegin()
-          for _i1150 in xrange(_size1146):
-            _elem1151 = Partition()
-            _elem1151.read(iprot)
-            self.success.append(_elem1151)
+          (_etype1170, _size1167) = iprot.readListBegin()
+          for _i1171 in xrange(_size1167):
+            _elem1172 = Partition()
+            _elem1172.read(iprot)
+            self.success.append(_elem1172)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27834,8 +27892,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 iter1152 in self.success:
-        iter1152.write(oprot)
+      for iter1173 in self.success:
+        iter1173.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27916,10 +27974,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1156, _size1153) = iprot.readListBegin()
-          for _i1157 in xrange(_size1153):
-            _elem1158 = iprot.readString()
-            self.part_vals.append(_elem1158)
+          (_etype1177, _size1174) = iprot.readListBegin()
+          for _i1178 in xrange(_size1174):
+            _elem1179 = iprot.readString()
+            self.part_vals.append(_elem1179)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27949,8 +28007,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 iter1159 in self.part_vals:
-        oprot.writeString(iter1159)
+      for iter1180 in self.part_vals:
+        oprot.writeString(iter1180)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28014,10 +28072,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1163, _size1160) = iprot.readListBegin()
-          for _i1164 in xrange(_size1160):
-            _elem1165 = iprot.readString()
-            self.success.append(_elem1165)
+          (_etype1184, _size1181) = iprot.readListBegin()
+          for _i1185 in xrange(_size1181):
+            _elem1186 = iprot.readString()
+            self.success.append(_elem1186)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28046,8 +28104,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 iter1166 in self.success:
-        oprot.writeString(iter1166)
+      for iter1187 in self.success:
+        oprot.writeString(iter1187)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28218,11 +28276,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1170, _size1167) = iprot.readListBegin()
-          for _i1171 in xrange(_size1167):
-            _elem1172 = Partition()
-            _elem1172.read(iprot)
-            self.success.append(_elem1172)
+          (_etype1191, _size1188) = iprot.readListBegin()
+          for _i1192 in xrange(_size1188):
+            _elem1193 = Partition()
+            _elem1193.read(iprot)
+            self.success.append(_elem1193)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28251,8 +28309,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 iter1173 in self.success:
-        iter1173.write(oprot)
+      for iter1194 in self.success:
+        iter1194.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28423,11 +28481,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1177, _size1174) = iprot.readListBegin()
-          for _i1178 in xrange(_size1174):
-            _elem1179 = PartitionSpec()
-            _elem1179.read(iprot)
-            self.success.append(_elem1179)
+          (_etype1198, _size1195) = iprot.readListBegin()
+          for _i1199 in xrange(_size1195):
+            _elem1200 = PartitionSpec()
+            _elem1200.read(iprot)
+            self.success.append(_elem1200)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28456,8 +28514,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 iter1180 in self.success:
-        iter1180.write(oprot)
+      for iter1201 in self.success:
+        iter1201.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28877,10 +28935,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1184, _size1181) = iprot.readListBegin()
-          for _i1185 in xrange(_size1181):
-            _elem1186 = iprot.readString()
-            self.names.append(_elem1186)
+          (_etype1205, _size1202) = iprot.readListBegin()
+          for _i1206 in xrange(_size1202):
+            _elem1207 = iprot.readString()
+            self.names.append(_elem1207)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28905,8 +28963,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 iter1187 in self.names:
-        oprot.writeString(iter1187)
+      for iter1208 in self.names:
+        oprot.writeString(iter1208)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28965,11 +29023,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1191, _size1188) = iprot.readListBegin()
-          for _i1192 in xrange(_size1188):
-            _elem1193 = Partition()
-            _elem1193.read(iprot)
-            self.success.append(_elem1193)
+          (_etype1212, _size1209) = iprot.readListBegin()
+          for _i1213 in xrange(_size1209):
+            _elem1214 = Partition()
+            _elem1214.read(iprot)
+            self.success.append(_elem1214)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28998,8 +29056,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 iter1194 in self.success:
-        iter1194.write(oprot)
+      for iter1215 in self.success:
+        iter1215.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29249,11 +29307,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1198, _size1195) = iprot.readListBegin()
-          for _i1199 in xrange(_size1195):
-            _elem1200 = Partition()
-            _elem1200.read(iprot)
-            self.new_parts.append(_elem1200)
+          (_etype1219, _size1216) = iprot.readListBegin()
+          for _i1220 in xrange(_size1216):
+            _elem1221 = Partition()
+            _elem1221.read(iprot)
+            self.new_parts.append(_elem1221)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29278,8 +29336,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 iter1201 in self.new_parts:
-        iter1201.write(oprot)
+      for iter1222 in self.new_parts:
+        iter1222.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29432,11 +29490,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1205, _size1202) = iprot.readListBegin()
-          for _i1206 in xrange(_size1202):
-            _elem1207 = Partition()
-            _elem1207.read(iprot)
-            self.new_parts.append(_elem1207)
+          (_etype1226, _size1223) = iprot.readListBegin()
+          for _i1227 in xrange(_size1223):
+            _elem1228 = Partition()
+            _elem1228.read(iprot)
+            self.new_parts.append(_elem1228)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29467,8 +29525,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1208 in self.new_parts:
-        iter1208.write(oprot)
+      for iter1229 in self.new_parts:
+        iter1229.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -29812,10 +29870,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1212, _size1209) = iprot.readListBegin()
-          for _i1213 in xrange(_size1209):
-            _elem1214 = iprot.readString()
-            self.part_vals.append(_elem1214)
+          (_etype1233, _size1230) = iprot.readListBegin()
+          for _i1234 in xrange(_size1230):
+            _elem1235 = iprot.readString()
+            self.part_vals.append(_elem1235)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29846,8 +29904,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 iter1215 in self.part_vals:
-        oprot.writeString(iter1215)
+      for iter1236 in self.part_vals:
+        oprot.writeString(iter1236)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -29989,10 +30047,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1219, _size1216) = iprot.readListBegin()
-          for _i1220 in xrange(_size1216):
-            _elem1221 = iprot.readString()
-            self.part_vals.append(_elem1221)
+          (_etype1240, _size1237) = iprot.readListBegin()
+          for _i1241 in xrange(_size1237):
+            _elem1242 = iprot.readString()
+            self.part_vals.append(_elem1242)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30014,8 +30072,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 iter1222 in self.part_vals:
-        oprot.writeString(iter1222)
+      for iter1243 in self.part_vals:
+        oprot.writeString(iter1243)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -30373,10 +30431,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1226, _size1223) = iprot.readListBegin()
-          for _i1227 in xrange(_size1223):
-            _elem1228 = iprot.readString()
-            self.success.append(_elem1228)
+          (_etype1247, _size1244) = iprot.readListBegin()
+          for _i1248 in xrange(_size1244):
+            _elem1249 = iprot.readString()
+            self.success.append(_elem1249)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30399,8 +30457,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 iter1229 in self.success:
-        oprot.writeString(iter1229)
+      for iter1250 in self.success:
+        oprot.writeString(iter1250)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30524,11 +30582,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1231, _vtype1232, _size1230 ) = iprot.readMapBegin()
-          for _i1234 in xrange(_size1230):
-            _key1235 = iprot.readString()
-            _val1236 = iprot.readString()
-            self.success[_key1235] = _val1236
+          (_ktype1252, _vtype1253, _size1251 ) = iprot.readMapBegin()
+          for _i1255 in xrange(_size1251):
+            _key1256 = iprot.readString()
+            _val1257 = iprot.readString()
+            self.success[_key1256] = _val1257
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30551,9 +30609,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 kiter1237,viter1238 in self.success.items():
-        oprot.writeString(kiter1237)
-        oprot.writeString(viter1238)
+      for kiter1258,viter1259 in self.success.items():
+        oprot.writeString(kiter1258)
+        oprot.writeString(viter1259)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30629,11 +30687,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1240, _vtype1241, _size1239 ) = iprot.readMapBegin()
-          for _i1243 in xrange(_size1239):
-            _key1244 = iprot.readString()
-            _val1245 = iprot.readString()
-            self.part_vals[_key1244] = _val1245
+          (_ktype1261, _vtype1262, _size1260 ) = iprot.readMapBegin()
+          for _i1264 in xrange(_size1260):
+            _key1265 = iprot.readString()
+            _val1266 = iprot.readString()
+            self.part_vals[_key1265] = _val1266
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30663,9 +30721,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 kiter1246,viter1247 in self.part_vals.items():
-        oprot.writeString(kiter1246)
-        oprot.writeString(viter1247)
+      for kiter1267,viter1268 in self.part_vals.items():
+        oprot.writeString(kiter1267)
+        oprot.writeString(viter1268)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -30879,11 +30937,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1249, _vtype1250, _size1248 ) = iprot.readMapBegin()
-          for _i1252 in xrange(_size1248):
-            _key1253 = iprot.readString()
-            _val1254 = iprot.readString()
-            self.part_vals[_key1253] = _val1254
+          (_ktype1270, _vtype1271, _size1269 ) = iprot.readMapBegin()
+          for _i1273 in xrange(_size1269):
+            _key1274 = iprot.readString()
+            _val1275 = iprot.readString()
+            self.part_vals[_key1274] = _val1275
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30913,9 +30971,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 kiter1255,viter1256 in self.part_vals.items():
-        oprot.writeString(kiter1255)
-        oprot.writeString(viter1256)
+      for kiter1276,viter1277 in self.part_vals.items():
+        oprot.writeString(kiter1276)
+        oprot.writeString(viter1277)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -34567,10 +34625,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1260, _size1257) = iprot.readListBegin()
-          for _i1261 in xrange(_size1257):
-            _elem1262 = iprot.readString()
-            self.success.append(_elem1262)
+          (_etype1281, _size1278) = iprot.readListBegin()
+          for _i1282 in xrange(_size1278):
+            _elem1283 = iprot.readString()
+            self.success.append(_elem1283)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34593,8 +34651,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 iter1263 in self.success:
-        oprot.writeString(iter1263)
+      for iter1284 in self.success:
+        oprot.writeString(iter1284)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35282,10 +35340,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1267, _size1264) = iprot.readListBegin()
-          for _i1268 in xrange(_size1264):
-            _elem1269 = iprot.readString()
-            self.success.append(_elem1269)
+          (_etype1288, _size1285) = iprot.readListBegin()
+          for _i1289 in xrange(_size1285):
+            _elem1290 = iprot.readString()
+            self.success.append(_elem1290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35308,8 +35366,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 iter1270 in self.success:
-        oprot.writeString(iter1270)
+      for iter1291 in self.success:
+        oprot.writeString(iter1291)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35823,11 +35881,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1274, _size1271) = iprot.readListBegin()
-          for _i1275 in xrange(_size1271):
-            _elem1276 = Role()
-            _elem1276.read(iprot)
-            self.success.append(_elem1276)
+          (_etype1295, _size1292) = iprot.readListBegin()
+          for _i1296 in xrange(_size1292):
+            _elem1297 = Role()
+            _elem1297.read(iprot)
+            self.success.append(_elem1297)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35850,8 +35908,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 iter1277 in self.success:
-        iter1277.write(oprot)
+      for iter1298 in self.success:
+        iter1298.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36360,10 +36418,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1281, _size1278) = iprot.readListBegin()
-          for _i1282 in xrange(_size1278):
-            _elem1283 = iprot.readString()
-            self.group_names.append(_elem1283)
+          (_etype1302, _size1299) = iprot.readListBegin()
+          for _i1303 in xrange(_size1299):
+            _elem1304 = iprot.readString()
+            self.group_names.append(_elem1304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36388,8 +36446,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 iter1284 in self.group_names:
-        oprot.writeString(iter1284)
+      for iter1305 in self.group_names:
+        oprot.writeString(iter1305)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36616,11 +36674,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1288, _size1285) = iprot.readListBegin()
-          for _i1289 in xrange(_size1285):
-            _elem1290 = HiveObjectPrivilege()
-            _elem1290.read(iprot)
-            self.success.append(_elem1290)
+          (_etype1309, _size1306) = iprot.readListBegin()
+          for _i1310 in xrange(_size1306):
+            _elem1311 = HiveObjectPrivilege()
+            _elem1311.read(iprot)
+            self.success.append(_elem1311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36643,8 +36701,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 iter1291 in self.success:
-        iter1291.write(oprot)
+      for iter1312 in self.success:
+        iter1312.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37314,10 +37372,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1295, _size1292) = iprot.readListBegin()
-          for _i1296 in xrange(_size1292):
-            _elem1297 = iprot.readString()
-            self.group_names.append(_elem1297)
+          (_etype1316, _size1313) = iprot.readListBegin()
+          for _i1317 in xrange(_size1313):
+            _elem1318 = iprot.readString()
+            self.group_names.append(_elem1318)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37338,8 +37396,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 iter1298 in self.group_names:
-        oprot.writeString(iter1298)
+      for iter1319 in self.group_names:
+        oprot.writeString(iter1319)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -37394,10 +37452,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1302, _size1299) = iprot.readListBegin()
-          for _i1303 in xrange(_size1299):
-            _elem1304 = iprot.readString()
-            self.success.append(_elem1304)
+          (_etype1323, _size1320) = iprot.readListBegin()
+          for _i1324 in xrange(_size1320):
+            _elem1325 = iprot.readString()
+            self.success.append(_elem1325)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37420,8 +37478,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 iter1305 in self.success:
-        oprot.writeString(iter1305)
+      for iter1326 in self.success:
+        oprot.writeString(iter1326)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -38353,10 +38411,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1309, _size1306) = iprot.readListBegin()
-          for _i1310 in xrange(_size1306):
-            _elem1311 = iprot.readString()
-            self.success.append(_elem1311)
+          (_etype1330, _size1327) = iprot.readListBegin()
+          for _i1331 in xrange(_size1327):
+            _elem1332 = iprot.readString()
+            self.success.append(_elem1332)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38373,8 +38431,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1312 in self.success:
-        oprot.writeString(iter1312)
+      for iter1333 in self.success:
+        oprot.writeString(iter1333)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38901,10 +38959,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1316, _size1313) = iprot.readListBegin()
-          for _i1317 in xrange(_size1313):
-            _elem1318 = iprot.readString()
-            self.success.append(_elem1318)
+          (_etype1337, _size1334) = iprot.readListBegin()
+          for _i1338 in xrange(_size1334):
+            _elem1339 = iprot.readString()
+            self.success.append(_elem1339)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38921,8 +38979,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1319 in self.success:
-        oprot.writeString(iter1319)
+      for iter1340 in self.success:
+        oprot.writeString(iter1340)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -42171,6 +42229,133 @@ class flushCache_result:
   def __ne__(self, other):
     return not (self == other)
 
+class add_write_notification_log_args:
+  """
+  Attributes:
+   - rqst
+  """
+
+  thrift_spec = None
+  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 = WriteNotificationLogRequest()
+          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('add_write_notification_log_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 add_write_notification_log_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (WriteNotificationLogResponse, WriteNotificationLogResponse.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = WriteNotificationLogResponse()
+          self.success.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('add_write_notification_log_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    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 cm_recycle_args:
   """
   Attributes:
@@ -47202,11 +47387,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1323, _size1320) = iprot.readListBegin()
-          for _i1324 in xrange(_size1320):
-            _elem1325 = SchemaVersion()
-            _elem1325.read(iprot)
-            self.success.append(_elem1325)
+          (_etype1344, _size1341) = iprot.readListBegin()
+          for _i1345 in xrange(_size1341):
+            _elem1346 = SchemaVersion()
+            _elem1346.read(iprot)
+            self.success.append(_elem1346)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -47235,8 +47420,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1326 in self.success:
-        iter1326.write(oprot)
+      for iter1347 in self.success:
+        iter1347.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -48711,11 +48896,11 @@ class get_runtime_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1330, _size1327) = iprot.readListBegin()
-          for _i1331 in xrange(_size1327):
-            _elem1332 = RuntimeStat()
-            _elem1332.read(iprot)
-            self.success.append(_elem1332)
+          (_etype1351, _size1348) = iprot.readListBegin()
+          for _i1352 in xrange(_size1348):
+            _elem1353 = RuntimeStat()
+            _elem1353.read(iprot)
+            self.success.append(_elem1353)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -48738,8 +48923,8 @@ class get_runtime_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1333 in self.success:
-        iter1333.write(oprot)
+      for iter1354 in self.success:
+        iter1354.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:


[16/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)


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

Branch: refs/heads/master
Commit: f519db7eafacb4b4d2d9fe2a9e10e908d8077224
Parents: 285a9b4
Author: Sankar Hariappan <sa...@apache.org>
Authored: Tue Jul 3 15:32:05 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Tue Jul 3 15:32:05 2018 +0530

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |  209 +-
 .../listener/DummyRawStoreFailEvent.java        |   15 +
 .../listener/TestDbNotificationListener.java    |    5 +
 .../hive/ql/parse/TestReplicationScenarios.java |   72 -
 .../TestReplicationScenariosAcidTables.java     |  602 ++-
 ...TestReplicationScenariosAcrossInstances.java |   15 +-
 .../hadoop/hive/ql/parse/WarehouseInstance.java |    5 +
 .../metastore/SynchronizedMetaStoreClient.java  |    5 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    6 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |    5 +-
 .../apache/hadoop/hive/ql/exec/ReplTxnTask.java |   31 +-
 .../hadoop/hive/ql/exec/repl/ReplDumpTask.java  |    4 +-
 .../IncrementalLoadTasksBuilder.java            |   73 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   37 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   24 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   18 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |    7 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   10 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  167 +-
 .../hadoop/hive/ql/metadata/HiveUtils.java      |   11 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   83 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    8 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   16 +-
 .../hadoop/hive/ql/parse/repl/CopyUtils.java    |    2 +-
 .../hadoop/hive/ql/parse/repl/dump/Utils.java   |    4 -
 .../repl/dump/events/CommitTxnHandler.java      |  125 +-
 .../ql/parse/repl/dump/events/EventHandler.java |   23 +-
 .../parse/repl/dump/events/InsertHandler.java   |    4 +
 .../parse/repl/load/UpdatedMetaDataTracker.java |  124 +-
 .../repl/load/message/AbortTxnHandler.java      |    7 +-
 .../repl/load/message/AllocWriteIdHandler.java  |    2 +-
 .../repl/load/message/CommitTxnHandler.java     |   78 +-
 .../parse/repl/load/message/MessageHandler.java |    8 +-
 .../parse/repl/load/message/OpenTxnHandler.java |    7 +-
 .../apache/hadoop/hive/ql/plan/MoveWork.java    |   12 +-
 .../apache/hadoop/hive/ql/plan/ReplTxnWork.java |   15 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2675 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 3905 ++++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  218 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../api/AllocateTableWriteIdsRequest.java       |   68 +-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/CommitTxnRequest.java    |  168 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |   32 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |   32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |  227 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../api/ReplTblWriteIdStateRequest.java         |   32 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |   32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 3468 ++++++++++------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../hive/metastore/api/WriteEventInfo.java      | 1012 +++++
 .../api/WriteNotificationLogRequest.java        |  949 +++++
 .../api/WriteNotificationLogResponse.java       |  283 ++
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1630 ++++----
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1630 +++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1139 ++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  933 +++--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   86 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   86 +
 .../hive/metastore/HiveMetaStoreClient.java     |   10 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +-
 .../hive/metastore/MetaStoreEventListener.java  |   12 +
 .../metastore/MetaStoreListenerNotifier.java    |    6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   60 +
 .../apache/hadoop/hive/metastore/RawStore.java  |   14 +
 .../hive/metastore/ReplChangeManager.java       |   10 +-
 .../hive/metastore/cache/CachedStore.java       |   12 +
 .../hive/metastore/events/AcidWriteEvent.java   |   91 +
 .../metastore/messaging/AcidWriteMessage.java   |   50 +
 .../metastore/messaging/CommitTxnMessage.java   |   23 +
 .../hive/metastore/messaging/EventMessage.java  |    3 +-
 .../messaging/MessageDeserializer.java          |    9 +
 .../metastore/messaging/MessageFactory.java     |   12 +
 .../messaging/json/JSONAcidWriteMessage.java    |  150 +
 .../messaging/json/JSONCommitTxnMessage.java    |   95 +
 .../messaging/json/JSONMessageDeserializer.java |    9 +
 .../messaging/json/JSONMessageFactory.java      |    8 +
 .../model/MTxnWriteNotificationLog.java         |  123 +
 .../hive/metastore/tools/SQLGenerator.java      |    9 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   28 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  187 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   11 +
 .../hadoop/hive/metastore/utils/FileUtils.java  |   12 +-
 .../src/main/resources/package.jdo              |   35 +
 .../main/sql/derby/hive-schema-3.1.0.derby.sql  |   15 +
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |   15 +
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |    1 -
 .../sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql  |   16 +
 .../main/sql/mssql/hive-schema-3.1.0.mssql.sql  |   17 +
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |   17 +
 .../sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql  |   16 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |    1 -
 .../main/sql/mysql/hive-schema-3.1.0.mysql.sql  |   16 +
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |   16 +
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |    4 +-
 .../sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql  |   16 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |    1 -
 .../sql/oracle/hive-schema-3.1.0.oracle.sql     |   15 +
 .../sql/oracle/hive-schema-4.0.0.oracle.sql     |   15 +
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |    4 +-
 .../oracle/upgrade-3.0.0-to-3.1.0.oracle.sql    |   16 +
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |    2 -
 .../sql/postgres/hive-schema-3.1.0.postgres.sql |   15 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |   15 +
 .../upgrade-3.0.0-to-3.1.0.postgres.sql         |   16 +
 .../src/main/thrift/hive_metastore.thrift       |   30 +-
 .../DummyRawStoreControlledCommit.java          |   11 +
 .../DummyRawStoreForJdoConnection.java          |   10 +
 .../HiveMetaStoreClientPreCatalog.java          |   10 +-
 137 files changed, 15896 insertions(+), 7205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 6321f9b..717cc8a 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -23,6 +23,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -75,11 +76,14 @@ import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
 import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -269,10 +273,16 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     public PartitionFiles next() {
       try {
         Partition p = partitionIter.next();
-        List<String> files = Lists.newArrayList(new FileIterator(p.getSd().getLocation()));
+        Iterator<String> fileIterator;
+        //For transactional tables, the actual file copy will be done by acid write event during replay of commit txn.
+        if (!TxnUtils.isTransactionalTable(t)) {
+          List<String> files = Lists.newArrayList(new FileIterator(p.getSd().getLocation()));
+          fileIterator = files.iterator();
+        } else {
+          fileIterator = Collections.emptyIterator();
+        }
         PartitionFiles partitionFiles =
-            new PartitionFiles(Warehouse.makePartName(t.getPartitionKeys(), p.getValues()),
-            files.iterator());
+            new PartitionFiles(Warehouse.makePartName(t.getPartitionKeys(), p.getValues()), fileIterator);
         return partitionFiles;
       } catch (MetaException e) {
         throw new RuntimeException(e);
@@ -414,10 +424,15 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
   class FileChksumIterator implements Iterator<String> {
     private List<String> files;
     private List<String> chksums;
+    private List<String> subDirs;
     int i = 0;
     FileChksumIterator(List<String> files, List<String> chksums) {
+      this(files, chksums, null);
+    }
+    FileChksumIterator(List<String> files, List<String> chksums, List<String> subDirs) {
       this.files = files;
       this.chksums = chksums;
+      this.subDirs = subDirs;
     }
     @Override
     public boolean hasNext() {
@@ -428,7 +443,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     public String next() {
       String result;
       try {
-        result = ReplChangeManager.encodeFileUri(files.get(i), chksums != null ? chksums.get(i) : null, null);
+        result = ReplChangeManager.encodeFileUri(files.get(i), chksums != null ? chksums.get(i) : null,
+                subDirs != null ? subDirs.get(i) : null);
       } catch (IOException e) {
         // File operations failed
         LOG.error("Encoding file URI failed with error " + e.getMessage());
@@ -623,6 +639,23 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     }
   }
 
+  @Override
+  public void onAcidWrite(AcidWriteEvent acidWriteEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+    AcidWriteMessage msg = msgFactory.buildAcidWriteMessage(acidWriteEvent,
+            new FileChksumIterator(acidWriteEvent.getFiles(), acidWriteEvent.getChecksums(),
+                    acidWriteEvent.getSubDirs()));
+    NotificationEvent event = new NotificationEvent(0, now(), EventType.ACID_WRITE.toString(), msg.toString());
+    event.setMessageFormat(msgFactory.getMessageFormat());
+    event.setDbName(acidWriteEvent.getDatabase());
+    event.setTableName(acidWriteEvent.getTable());
+    try {
+      addWriteNotificationLog(event, acidWriteEvent, dbConn, sqlGenerator, msg);
+    } catch (SQLException e) {
+      throw new MetaException("Unable to add write notification log " + StringUtils.stringifyException(e));
+    }
+  }
+
   private int now() {
     long millis = System.currentTimeMillis();
     millis /= 1000;
@@ -634,12 +667,133 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     return (int)millis;
   }
 
+  /**
+   * Close statement instance.
+   * @param stmt statement instance.
+   */
+  private static void closeStmt(Statement stmt) {
+    try {
+      if (stmt != null && !stmt.isClosed()) {
+        stmt.close();
+      }
+    } catch (SQLException e) {
+      LOG.warn("Failed to close statement " + e.getMessage());
+    }
+  }
+
+  /**
+   * Close the ResultSet.
+   * @param rs may be {@code null}
+   */
+  private static void close(ResultSet rs) {
+    try {
+      if (rs != null && !rs.isClosed()) {
+        rs.close();
+      }
+    } catch(SQLException ex) {
+      LOG.warn("Failed to close result set " + ex.getMessage());
+    }
+  }
+
+  private long getNextNLId(Statement stmt, SQLGenerator sqlGenerator, String sequence)
+          throws SQLException, MetaException {
+    String s = sqlGenerator.addForUpdateClause("select \"NEXT_VAL\" from " +
+            "\"SEQUENCE_TABLE\" where \"SEQUENCE_NAME\" = " + quoteString(sequence));
+    LOG.debug("Going to execute query <" + s + ">");
+    ResultSet rs = null;
+    try {
+      rs = stmt.executeQuery(s);
+      if (!rs.next()) {
+        throw new MetaException("Transaction database not properly configured, can't find next NL id.");
+      }
+
+      long nextNLId = rs.getLong(1);
+      long updatedNLId = nextNLId + 1;
+      s = "update \"SEQUENCE_TABLE\" set \"NEXT_VAL\" = " + updatedNLId + " where \"SEQUENCE_NAME\" = " +
+              quoteString(sequence);
+      LOG.debug("Going to execute update <" + s + ">");
+      stmt.executeUpdate(s);
+      return nextNLId;
+    }finally {
+      close(rs);
+    }
+  }
+
+  private void addWriteNotificationLog(NotificationEvent event, AcidWriteEvent acidWriteEvent, Connection dbConn,
+                                 SQLGenerator sqlGenerator, AcidWriteMessage msg) throws MetaException, SQLException {
+    LOG.debug("DbNotificationListener: adding write notification log for : {}", event.getMessage());
+    assert ((dbConn != null) && (sqlGenerator != null));
+
+    Statement stmt =null;
+    ResultSet rs = null;
+    String dbName = acidWriteEvent.getDatabase();
+    String tblName = acidWriteEvent.getTable();
+    String partition = acidWriteEvent.getPartition();
+    String tableObj = msg.getTableObjStr();
+    String partitionObj = msg.getPartitionObjStr();
+    String files = ReplChangeManager.joinWithSeparator(msg.getFiles());
+
+    try {
+      stmt = dbConn.createStatement();
+      if (sqlGenerator.getDbProduct() == MYSQL) {
+        stmt.execute("SET @@session.sql_mode=ANSI_QUOTES");
+      }
+
+      String s = sqlGenerator.addForUpdateClause("select \"WNL_FILES\", \"WNL_ID\" from" +
+                      " \"TXN_WRITE_NOTIFICATION_LOG\" " +
+                      "where \"WNL_DATABASE\" = " + quoteString(dbName) +
+                      "and \"WNL_TABLE\" = " + quoteString(tblName) +  " and \"WNL_PARTITION\" = " +
+                      quoteString(partition) + " and \"WNL_TXNID\" = " + Long.toString(acidWriteEvent.getTxnId()));
+      LOG.debug("Going to execute query <" + s + ">");
+      rs = stmt.executeQuery(s);
+      if (!rs.next()) {
+        // if rs is empty then no lock is taken and thus it can not cause deadlock.
+        long nextNLId = getNextNLId(stmt, sqlGenerator,
+                "org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog");
+        s = "insert into \"TXN_WRITE_NOTIFICATION_LOG\" (\"WNL_ID\", \"WNL_TXNID\", \"WNL_WRITEID\"," +
+                " \"WNL_DATABASE\", \"WNL_TABLE\"," +
+                " \"WNL_PARTITION\", \"WNL_TABLE_OBJ\", \"WNL_PARTITION_OBJ\", \"WNL_FILES\", \"WNL_EVENT_TIME\")" +
+                " values (" + nextNLId
+                + "," + acidWriteEvent.getTxnId() +  "," + acidWriteEvent.getWriteId()+  "," +
+                quoteString(dbName)+  "," +  quoteString(tblName)+  "," + quoteString(partition)+  "," +
+                quoteString(tableObj)+  "," + quoteString(partitionObj) +  "," +  quoteString(files)+
+                "," +  now() + ")";
+        LOG.info("Going to execute insert <" + s + ">");
+        stmt.execute(sqlGenerator.addEscapeCharacters(s));
+      } else {
+        String existingFiles = rs.getString(1);
+        if (existingFiles.contains(sqlGenerator.addEscapeCharacters(files))) {
+          // If list of files are already present then no need to update it again. This scenario can come in case of
+          // retry done to the meta store for the same operation.
+          LOG.info("file list " + files + " already present");
+          return;
+        }
+        long nlId = rs.getLong(2);
+        files = ReplChangeManager.joinWithSeparator(Lists.newArrayList(files, existingFiles));
+        s = "update \"TXN_WRITE_NOTIFICATION_LOG\" set \"WNL_TABLE_OBJ\" = " +  quoteString(tableObj) + "," +
+                " \"WNL_PARTITION_OBJ\" = " + quoteString(partitionObj) + "," +
+                " \"WNL_FILES\" = " + quoteString(files) + "," +
+                " \"WNL_EVENT_TIME\" = " + now() +
+                " where \"WNL_ID\" = " + nlId;
+        LOG.info("Going to execute update <" + s + ">");
+        stmt.executeUpdate(sqlGenerator.addEscapeCharacters(s));
+      }
+    } catch (SQLException e) {
+      LOG.warn("failed to add write notification log" + e.getMessage());
+      throw e;
+    } finally {
+      closeStmt(stmt);
+      close(rs);
+    }
+  }
+
   static String quoteString(String input) {
     return "'" + input + "'";
   }
 
   private void addNotificationLog(NotificationEvent event, ListenerEvent listenerEvent, Connection dbConn,
                                   SQLGenerator sqlGenerator) throws MetaException, SQLException {
+    LOG.debug("DbNotificationListener: adding notification log for : {}", event.getMessage());
     if ((dbConn == null) || (sqlGenerator == null)) {
       LOG.info("connection or sql generator is not set so executing sql via DN");
       process(event, listenerEvent);
@@ -669,22 +823,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       LOG.debug("Going to execute update <" + s + ">");
       stmt.executeUpdate(s);
 
-      s = sqlGenerator.addForUpdateClause("select \"NEXT_VAL\" from " +
-              "\"SEQUENCE_TABLE\" where \"SEQUENCE_NAME\" = " +
-              " 'org.apache.hadoop.hive.metastore.model.MNotificationLog'");
-      LOG.debug("Going to execute query <" + s + ">");
-      rs = stmt.executeQuery(s);
-      if (!rs.next()) {
-        throw new MetaException("failed to get next NEXT_VAL from SEQUENCE_TABLE");
-      }
-
-      long nextNLId = rs.getLong(1);
-      long updatedNLId = nextNLId + 1;
-      s = "update \"SEQUENCE_TABLE\" set \"NEXT_VAL\" = " + updatedNLId + " where \"SEQUENCE_NAME\" = " +
-
-              " 'org.apache.hadoop.hive.metastore.model.MNotificationLog'";
-      LOG.debug("Going to execute update <" + s + ">");
-      stmt.executeUpdate(s);
+      long nextNLId = getNextNLId(stmt, sqlGenerator,
+              "org.apache.hadoop.hive.metastore.model.MNotificationLog");
 
       List<String> insert = new ArrayList<>();
 
@@ -712,20 +852,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       LOG.warn("failed to add notification log" + e.getMessage());
       throw e;
     } finally {
-      if (stmt != null && !stmt.isClosed()) {
-        try {
-          stmt.close();
-        } catch (SQLException e) {
-          LOG.warn("Failed to close statement " + e.getMessage());
-        }
-      }
-      if (rs != null && !rs.isClosed()) {
-        try {
-          rs.close();
-        } catch (SQLException e) {
-          LOG.warn("Failed to close result set " + e.getMessage());
-        }
-      }
+      closeStmt(stmt);
+      close(rs);
     }
   }
 
@@ -742,12 +870,12 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         event.getMessage());
     HMSHandler.getMSForConf(conf).addNotificationEvent(event);
 
-      // Set the DB_NOTIFICATION_EVENT_ID for future reference by other listeners.
-      if (event.isSetEventId()) {
-        listenerEvent.putParameter(
-            MetaStoreEventListenerConstants.DB_NOTIFICATION_EVENT_ID_KEY_NAME,
-            Long.toString(event.getEventId()));
-      }
+    // Set the DB_NOTIFICATION_EVENT_ID for future reference by other listeners.
+    if (event.isSetEventId()) {
+      listenerEvent.putParameter(
+          MetaStoreEventListenerConstants.DB_NOTIFICATION_EVENT_ID_KEY_NAME,
+          Long.toString(event.getEventId()));
+    }
   }
 
   private static class CleanerThread extends Thread {
@@ -768,6 +896,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       while (true) {
         try {
           rs.cleanNotificationEvents(ttl);
+          rs.cleanWriteNotificationEvents(ttl);
         } catch (Exception ex) {
           //catching exceptions here makes sure that the thread doesn't die in case of unexpected
           //exceptions

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index abf67a8..b4b118e 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -880,6 +881,20 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    if (!shouldEventSucceed) {
+      //throw exception to simulate an issue with cleaner thread
+      throw new RuntimeException("Dummy exception while cleaning write notifications");
+    }
+    objectStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return objectStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
+  @Override
   public CurrentNotificationEventId getCurrentNotificationEventId() {
     return objectStore.getCurrentNotificationEventId();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index eef917e..82429e3 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
@@ -238,6 +239,10 @@ public class TestDbNotificationListener {
     public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent) throws MetaException {
       pushEventId(EventType.ALLOC_WRITE_ID, allocWriteIdEvent);
     }
+
+    public void onAcidWrite(AcidWriteEvent acidWriteEvent) throws MetaException {
+      pushEventId(EventType.ACID_WRITE, acidWriteEvent);
+    }
   }
 
   @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 46c623d..c82a933 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -2833,78 +2833,6 @@ public class TestReplicationScenarios {
     verifyRun("SELECT max(a) from " + replDbName + ".ptned2 where b=1", new String[]{"8"}, driverMirror);
   }
 
-  // TODO: This test should be removed once ACID tables replication is supported.
-  @Test
-  public void testSkipTables() throws Exception {
-    String testName = "skipTables";
-    String dbName = createDB(testName, driver);
-    String replDbName = dbName + "_dupe";
-
-    // TODO: this is wrong; this test sets up dummy txn manager and so it cannot create ACID tables.
-    //       If I change it to use proper txn manager, the setup for some tests hangs.
-    //       This used to work by accident, now this works due a test flag. The test needs to be fixed.
-    // Create table
-    run("CREATE TABLE " + dbName + ".acid_table (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-    run("CREATE TABLE " + dbName + ".mm_table (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true'," +
-        " 'transactional_properties'='insert_only')", driver);
-    verifyIfTableExist(dbName, "acid_table", metaStoreClient);
-    verifyIfTableExist(dbName, "mm_table", metaStoreClient);
-
-    // Bootstrap test
-    Tuple bootstrapDump = bootstrapLoadAndVerify(dbName, replDbName);
-    String replDumpId = bootstrapDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table", metaStoreClientMirror);
-    verifyIfTableNotExist(replDbName, "mm_table", metaStoreClientMirror);
-
-    // Test alter table
-    run("ALTER TABLE " + dbName + ".acid_table RENAME TO " + dbName + ".acid_table_rename", driver);
-    verifyIfTableExist(dbName, "acid_table_rename", metaStoreClient);
-
-    // Dummy create table command to mark proper last repl ID after dump
-    run("CREATE TABLE " + dbName + ".dummy (a int)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    Tuple incrementalDump = incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    replDumpId = incrementalDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table_rename", metaStoreClientMirror);
-
-    // Create another table for incremental repl verification
-    run("CREATE TABLE " + dbName + ".acid_table_incremental (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-    run("CREATE TABLE " + dbName + ".mm_table_incremental (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true'," +
-        " 'transactional_properties'='insert_only')", driver);
-    verifyIfTableExist(dbName, "acid_table_incremental", metaStoreClient);
-    verifyIfTableExist(dbName, "mm_table_incremental", metaStoreClient);
-
-    // Dummy insert into command to mark proper last repl ID after dump
-    run("INSERT INTO " + dbName + ".dummy values(1)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    incrementalDump = incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    replDumpId = incrementalDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table_incremental", metaStoreClientMirror);
-    verifyIfTableNotExist(replDbName, "mm_table_incremental", metaStoreClientMirror);
-
-    // Test adding a constraint
-    run("ALTER TABLE " + dbName + ".acid_table_incremental ADD CONSTRAINT key_pk PRIMARY KEY (key) DISABLE NOVALIDATE", driver);
-    try {
-      List<SQLPrimaryKey> pks = metaStoreClient.getPrimaryKeys(new PrimaryKeysRequest(dbName, "acid_table_incremental"));
-      assertEquals(pks.size(), 1);
-    } catch (TException te) {
-      assertNull(te);
-    }
-
-    // Dummy insert into command to mark proper last repl ID after dump
-    run("INSERT INTO " + dbName + ".dummy values(2)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    verifyIfTableNotExist(replDbName, "acid_table_incremental", metaStoreClientMirror);
-  }
-
   @Test
   public void testDeleteStagingDir() throws IOException {
     String testName = "deleteStagingDir";

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
index 86c0405..8c683cf 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.junit.rules.TestName;
+
 import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,6 +54,8 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import javax.annotation.Nullable;
+import java.util.Collections;
+import com.google.common.collect.Lists;
 
 /**
  * TestReplicationScenariosAcidTables - test replication for ACID tables
@@ -66,8 +69,13 @@ public class TestReplicationScenariosAcidTables {
 
   protected static final Logger LOG = LoggerFactory.getLogger(TestReplicationScenarios.class);
   private static WarehouseInstance primary, replica, replicaNonAcid;
-  private String primaryDbName, replicatedDbName;
   private static HiveConf conf;
+  private String primaryDbName, replicatedDbName, primaryDbNameExtra;
+  private enum OperationType {
+    REPL_TEST_ACID_INSERT, REPL_TEST_ACID_INSERT_SELECT, REPL_TEST_ACID_CTAS,
+    REPL_TEST_ACID_INSERT_OVERWRITE, REPL_TEST_ACID_INSERT_IMPORT, REPL_TEST_ACID_INSERT_LOADLOCAL,
+    REPL_TEST_ACID_INSERT_UNION
+  }
 
   @BeforeClass
   public static void classLevelSetup() throws Exception {
@@ -80,9 +88,13 @@ public class TestReplicationScenariosAcidTables {
         put("fs.defaultFS", miniDFSCluster.getFileSystem().getUri().toString());
         put("hive.support.concurrency", "true");
         put("hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
-        put("hive.repl.dump.include.acid.tables", "true");
         put("hive.metastore.client.capability.check", "false");
         put("hive.repl.bootstrap.dump.open.txn.timeout", "1s");
+        put("hive.exec.dynamic.partition.mode", "nonstrict");
+        put("hive.strict.checks.bucketing", "false");
+        put("hive.mapred.mode", "nonstrict");
+        put("mapred.input.dir.recursive", "true");
+        put("hive.metastore.disallow.incompatible.col.type.changes", "false");
     }};
     primary = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
     replica = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
@@ -90,7 +102,6 @@ public class TestReplicationScenariosAcidTables {
         put("fs.defaultFS", miniDFSCluster.getFileSystem().getUri().toString());
         put("hive.support.concurrency", "false");
         put("hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager");
-        put("hive.repl.dump.include.acid.tables", "true");
         put("hive.metastore.client.capability.check", "false");
     }};
     replicaNonAcid = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf1);
@@ -109,6 +120,9 @@ public class TestReplicationScenariosAcidTables {
     replicatedDbName = "replicated_" + primaryDbName;
     primary.run("create database " + primaryDbName + " WITH DBPROPERTIES ( '" +
             SOURCE_OF_REPLICATION + "' = '1,2,3')");
+    primaryDbNameExtra = primaryDbName+"_extra";
+    primary.run("create database " + primaryDbNameExtra + " WITH DBPROPERTIES ( '" +
+            SOURCE_OF_REPLICATION + "' = '1,2,3')");
   }
 
   @After
@@ -116,6 +130,7 @@ public class TestReplicationScenariosAcidTables {
     primary.run("drop database if exists " + primaryDbName + " cascade");
     replica.run("drop database if exists " + replicatedDbName + " cascade");
     replicaNonAcid.run("drop database if exists " + replicatedDbName + " cascade");
+    primary.run("drop database if exists " + primaryDbName + "_extra cascade");
   }
 
   @Test
@@ -482,4 +497,585 @@ public class TestReplicationScenariosAcidTables {
     primary.run("DROP TABLE " + dbName + ".normal");
     primary.run("drop database " + dbName);
   }
+
+  @Test
+  public void testAcidTableIncrementalReplication() throws Throwable {
+    WarehouseInstance.Tuple bootStrapDump = primary.dump(primaryDbName, null);
+    replica.load(replicatedDbName, bootStrapDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName)
+            .verifyResult(bootStrapDump.lastReplicationId);
+    List<String> selectStmtList = new ArrayList<>();
+    List<String[]> expectedValues = new ArrayList<>();
+
+    appendInsert(selectStmtList, expectedValues);
+    appendDelete(selectStmtList, expectedValues);
+    appendUpdate(selectStmtList, expectedValues);
+    appendTruncate(selectStmtList, expectedValues);
+    appendInsertIntoFromSelect(selectStmtList, expectedValues);
+    appendMerge(selectStmtList, expectedValues);
+    appendCreateAsSelect(selectStmtList, expectedValues);
+    appendImport(selectStmtList, expectedValues);
+    appendInsertOverwrite(selectStmtList, expectedValues);
+    //appendLoadLocal(selectStmtList, expectedValues);
+    appendInsertUnion(selectStmtList, expectedValues);
+    appendMultiStatementTxn(selectStmtList, expectedValues);
+    appendMultiStatementTxnUpdateDelete(selectStmtList, expectedValues);
+
+    verifyIncrementalLoad(selectStmtList, expectedValues, bootStrapDump.lastReplicationId);
+  }
+
+  private void appendInsert(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsert";
+    String tableNameMM = tableName + "_MM";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendDelete(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testDelete";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    deleteRecords(tableName);
+    selectStmtList.add("select count(*) from " + tableName);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  private void appendUpdate(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testUpdate";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    updateRecords(tableName);
+    selectStmtList.add("select value from " + tableName + " order by value");
+    expectedValues.add(new String[] {"1", "100", "100", "100", "100"});
+  }
+
+  private void appendTruncate(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testTruncate";
+    String tableNameMM = tableName + "_MM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    truncateTable(primaryDbName, tableName);
+    selectStmtList.add("select count(*) from " + tableName);
+    expectedValues.add(new String[] {"0"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    truncateTable(primaryDbName, tableNameMM);
+    selectStmtList.add("select count(*) from " + tableNameMM);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  private void appendInsertIntoFromSelect(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertIntoFromSelect";
+    String tableNameMM =tableName + "_MM";
+    String tableNameSelect = testName.getMethodName() + "_Select";
+    String tableNameSelectMM = testName.getMethodName() + "_SelectMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameSelect, false, OperationType.REPL_TEST_ACID_INSERT_SELECT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameSelect + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameSelectMM, true, OperationType.REPL_TEST_ACID_INSERT_SELECT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameSelectMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendMerge(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testMerge";
+    String tableNameMerge = testName.getMethodName() + "_Merge";
+
+    insertForMerge(tableName, tableNameMerge, false);
+    selectStmtList.add("select last_update_user from " + tableName + " order by last_update_user");
+    expectedValues.add(new String[] {"creation", "creation", "creation", "creation", "creation",
+            "creation", "creation", "merge_update", "merge_insert", "merge_insert"});
+    selectStmtList.add("select ID from " + tableNameMerge + " order by ID");
+    expectedValues.add(new String[] {"1", "4", "7", "8", "8", "11"});
+  }
+
+  private void appendCreateAsSelect(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testCreateAsSelect";
+    String tableNameMM = tableName + "_MM";
+    String tableNameCTAS = testName.getMethodName() + "_CTAS";
+    String tableNameCTASMM = testName.getMethodName() + "_CTASMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameCTAS, false, OperationType.REPL_TEST_ACID_CTAS);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameCTAS + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameCTASMM, true, OperationType.REPL_TEST_ACID_CTAS);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameCTASMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendImport(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testImport";
+    String tableNameMM = tableName + "_MM";
+    String tableNameImport = testName.getMethodName() + "_Import";
+    String tableNameImportMM = testName.getMethodName() + "_ImportMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameImport, false, OperationType.REPL_TEST_ACID_INSERT_IMPORT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameImport + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameImportMM, true, OperationType.REPL_TEST_ACID_INSERT_IMPORT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameImportMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendInsertOverwrite(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertOverwrite";
+    String tableNameOW = testName.getMethodName() +"_OW";
+    String tableNameMM = tableName + "_MM";
+    String tableNameOWMM = testName.getMethodName() +"_OWMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameOW, false, OperationType.REPL_TEST_ACID_INSERT_OVERWRITE);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameOW + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameOWMM, true, OperationType.REPL_TEST_ACID_INSERT_OVERWRITE);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameOWMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  //TODO: need to check why its failing. Loading to acid table from local path is failing.
+  private void appendLoadLocal(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testLoadLocal";
+    String tableNameLL = testName.getMethodName() +"_LL";
+    String tableNameMM = tableName + "_MM";
+    String tableNameLLMM = testName.getMethodName() +"_LLMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameLL, false, OperationType.REPL_TEST_ACID_INSERT_LOADLOCAL);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameLL + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameLLMM, true, OperationType.REPL_TEST_ACID_INSERT_LOADLOCAL);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameLLMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendInsertUnion(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertUnion";
+    String tableNameUnion = testName.getMethodName() +"_UNION";
+    String tableNameMM = tableName + "_MM";
+    String tableNameUnionMM = testName.getMethodName() +"_UNIONMM";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String[] resultArrayUnion = new String[]{"1", "1", "2", "2", "3", "3", "4", "4", "5", "5"};
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameUnion, false, OperationType.REPL_TEST_ACID_INSERT_UNION);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add( "select key from " + tableNameUnion + " order by key");
+    expectedValues.add(resultArrayUnion);
+    selectStmtList.add("select key from " + tableName + "_nopart" + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add("select key from " + tableNameUnion + "_nopart" + " order by key");
+    expectedValues.add(resultArrayUnion);
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameUnionMM, true, OperationType.REPL_TEST_ACID_INSERT_UNION);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add( "select key from " + tableNameUnionMM + " order by key");
+    expectedValues.add(resultArrayUnion);
+    selectStmtList.add("select key from " + tableNameMM + "_nopart" + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add("select key from " + tableNameUnionMM + "_nopart" + " order by key");
+    expectedValues.add(resultArrayUnion);
+  }
+
+  private void appendMultiStatementTxn(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testMultiStatementTxn";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableNameMM = tableName + "_MM";
+    String tableProperty = "'transactional'='true'";
+
+    insertIntoDB(primaryDbName, tableName, tableProperty, resultArray, true);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    tableProperty = setMMtableProperty(tableProperty);
+    insertIntoDB(primaryDbName, tableNameMM, tableProperty, resultArray, true);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendMultiStatementTxnUpdateDelete(List<String> selectStmtList, List<String[]> expectedValues)
+          throws Throwable {
+    String tableName = testName.getMethodName() + "testMultiStatementTxnUpdate";
+    String tableNameDelete = testName.getMethodName() + "testMultiStatementTxnDelete";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+
+    insertIntoDB(primaryDbName, tableName, tableProperty, resultArray, true);
+    updateRecords(tableName);
+    selectStmtList.add("select value from " + tableName + " order by value");
+    expectedValues.add(new String[] {"1", "100", "100", "100", "100"});
+
+    insertIntoDB(primaryDbName, tableNameDelete, tableProperty, resultArray, true);
+    deleteRecords(tableNameDelete);
+    selectStmtList.add("select count(*) from " + tableNameDelete);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  @Test
+  public void testReplCM() throws Throwable {
+    String tableName = testName.getMethodName();
+    String tableNameMM = testName.getMethodName() + "_MM";
+    String[] result = new String[]{"5"};
+
+    WarehouseInstance.Tuple incrementalDump;
+    WarehouseInstance.Tuple bootStrapDump = primary.dump(primaryDbName, null);
+    replica.load(replicatedDbName, bootStrapDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName)
+            .verifyResult(bootStrapDump.lastReplicationId);
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    incrementalDump = primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
+    truncateTable(primaryDbName, tableName);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(Lists.newArrayList("select count(*) from " + tableName,
+                                              "select count(*) from " + tableName + "_nopart"),
+                            Lists.newArrayList(result, result));
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    incrementalDump = primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
+    truncateTable(primaryDbName, tableNameMM);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(Lists.newArrayList("select count(*) from " + tableNameMM,
+            "select count(*) from " + tableNameMM + "_nopart"),
+            Lists.newArrayList(result, result));
+  }
+
+  @Test
+  public void testMultiDBTxn() throws Throwable {
+    String tableName = testName.getMethodName();
+    String dbName1 = tableName + "_db1";
+    String dbName2 = tableName + "_db2";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+    String txnStrStart = "START TRANSACTION";
+    String txnStrCommit = "COMMIT";
+
+    WarehouseInstance.Tuple incrementalDump;
+    primary.run("alter database default set dbproperties ('repl.source.for' = '1, 2, 3')");
+    WarehouseInstance.Tuple bootStrapDump = primary.dump("`*`", null);
+
+    primary.run("use " + primaryDbName)
+          .run("create database " + dbName1 + " WITH DBPROPERTIES ( '" + SOURCE_OF_REPLICATION + "' = '1,2,3')")
+          .run("create database " + dbName2 + " WITH DBPROPERTIES ( '" + SOURCE_OF_REPLICATION + "' = '1,2,3')")
+          .run("CREATE TABLE " + dbName1 + "." + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+                  "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+          .run("use " + dbName1)
+          .run("SHOW TABLES LIKE '" + tableName + "'")
+          .verifyResult(tableName)
+          .run("CREATE TABLE " + dbName2 + "." + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+                  "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+          .run("use " + dbName2)
+          .run("SHOW TABLES LIKE '" + tableName + "'")
+          .verifyResult(tableName)
+          .run(txnStrStart)
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+          .run("select key from " + dbName2 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run("select key from " + dbName1 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run(txnStrCommit);
+
+    incrementalDump = primary.dump("`*`", bootStrapDump.lastReplicationId);
+
+    // Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM
+    // we are not able to create multiple embedded derby instances for two different MetaStore instances.
+    primary.run("drop database " + primaryDbName + " cascade");
+    primary.run("drop database " + dbName1 + " cascade");
+    primary.run("drop database " + dbName2 + " cascade");
+    //End of additional steps
+
+    replica.loadWithoutExplain("", bootStrapDump.dumpLocation)
+            .run("REPL STATUS default")
+            .verifyResult(bootStrapDump.lastReplicationId);
+
+    replica.loadWithoutExplain("", incrementalDump.dumpLocation)
+          .run("REPL STATUS " + dbName1)
+          .run("select key from " + dbName1 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run("select key from " + dbName2 + "." + tableName + " order by key")
+          .verifyResults(resultArray);
+
+    replica.run("drop database " + primaryDbName + " cascade");
+    replica.run("drop database " + dbName1 + " cascade");
+    replica.run("drop database " + dbName2 + " cascade");
+  }
+
+  private void verifyResultsInReplica(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable  {
+    for (int idx = 0; idx < selectStmtList.size(); idx++) {
+      replica.run("use " + replicatedDbName)
+              .run(selectStmtList.get(idx))
+              .verifyResults(expectedValues.get(idx));
+    }
+  }
+
+  private WarehouseInstance.Tuple verifyIncrementalLoad(List<String> selectStmtList,
+                                                  List<String[]> expectedValues, String lastReplId) throws Throwable {
+    WarehouseInstance.Tuple incrementalDump = primary.dump(primaryDbName, lastReplId);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(selectStmtList, expectedValues);
+
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(selectStmtList, expectedValues);
+    return incrementalDump;
+  }
+
+  private void deleteRecords(String tableName) throws Throwable {
+    primary.run("use " + primaryDbName)
+            .run("delete from " + tableName)
+            .run("select count(*) from " + tableName)
+            .verifyResult("0");
+  }
+
+  private void updateRecords(String tableName) throws Throwable {
+    primary.run("use " + primaryDbName)
+            .run("update " + tableName + " set value = 100 where key >= 2")
+            .run("select value from " + tableName + " order by value")
+            .verifyResults(new String[] {"1", "100", "100", "100", "100"});
+  }
+
+  private void truncateTable(String dbName, String tableName) throws Throwable {
+    primary.run("use " + dbName)
+            .run("truncate table " + tableName)
+            .run("select count(*) from " + tableName)
+            .verifyResult("0")
+            .run("truncate table " + tableName + "_nopart")
+            .run("select count(*) from " + tableName + "_nopart")
+            .verifyResult("0");
+  }
+
+  private WarehouseInstance.Tuple verifyLoad(String tableName, String tableNameOp, String lastReplId) throws Throwable {
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    if (tableNameOp == null) {
+      return verifyIncrementalLoad(Lists.newArrayList("select key from " + tableName + " order by key",
+              "select key from " + tableName + "_nopart order by key"),
+              Lists.newArrayList(resultArray, resultArray), lastReplId);
+    }
+    return verifyIncrementalLoad(Lists.newArrayList("select key from " + tableName + " order by key",
+                                                    "select key from " + tableNameOp + " order by key",
+                                                    "select key from " + tableName + "_nopart" + " order by key",
+                                                    "select key from " + tableNameOp + "_nopart" + " order by key"),
+                    Lists.newArrayList(resultArray, resultArray, resultArray, resultArray), lastReplId);
+  }
+
+  private void insertIntoDB(String dbName, String tableName, String tableProperty, String[] resultArray, boolean isTxn)
+          throws Throwable {
+    String txnStrStart = "START TRANSACTION";
+    String txnStrCommit = "COMMIT";
+    if (!isTxn) {
+      txnStrStart = "use " + dbName; //dummy
+      txnStrCommit = "use " + dbName; //dummy
+    }
+    primary.run("use " + dbName);
+    primary.run("CREATE TABLE " + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+            "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+            .run("SHOW TABLES LIKE '" + tableName + "'")
+            .verifyResult(tableName)
+            .run("CREATE TABLE " + tableName + "_nopart (key int, value int) " +
+                    "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+            .run("SHOW TABLES LIKE '" + tableName + "_nopart'")
+            .run("ALTER TABLE " + tableName + " ADD PARTITION (load_date='2016-03-03')")
+            .run(txnStrStart)
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+            .run("select key from " + tableName + " order by key")
+            .verifyResults(resultArray)
+            .run("INSERT INTO " + tableName + "_nopart (key, value) select key, value from " + tableName)
+            .run("select key from " + tableName + "_nopart" + " order by key")
+            .verifyResults(resultArray)
+            .run(txnStrCommit);
+  }
+
+  private void insertIntoDB(String dbName, String tableName, String tableProperty, String[] resultArray)
+          throws Throwable {
+    insertIntoDB(dbName, tableName, tableProperty, resultArray, false);
+  }
+
+  private void insertRecords(String tableName, String tableNameOp, boolean isMMTable,
+                             OperationType opType) throws Throwable {
+    insertRecordsIntoDB(primaryDbName, tableName, tableNameOp, isMMTable, opType);
+  }
+
+  private void insertRecordsIntoDB(String DbName, String tableName, String tableNameOp, boolean isMMTable,
+                             OperationType opType) throws Throwable {
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+    if (isMMTable) {
+      tableProperty = setMMtableProperty(tableProperty);
+    }
+    primary.run("use " + DbName);
+
+    switch (opType) {
+      case REPL_TEST_ACID_INSERT:
+        insertIntoDB(DbName, tableName, tableProperty, resultArray);
+        insertIntoDB(primaryDbNameExtra, tableName, tableProperty, resultArray);
+        return;
+      case REPL_TEST_ACID_INSERT_OVERWRITE:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+              "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( "+ tableProperty + " )")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-01') VALUES (2, 2)")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-01') VALUES (10, 12)")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-02') VALUES (11, 1)")
+        .run("select key from " + tableNameOp + " order by key")
+        .verifyResults(new String[]{"2", "10", "11"})
+        .run("insert overwrite table " + tableNameOp + " select * from " + tableName)
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( "+ tableProperty + " )")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (2, 2)")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (10, 12)")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (11, 1)")
+        .run("select key from " + tableNameOp + "_nopart" + " order by key")
+        .verifyResults(new String[]{"2", "10", "11"})
+        .run("insert overwrite table " + tableNameOp + "_nopart select * from " + tableName + "_nopart")
+        .run("select key from " + tableNameOp + "_nopart" + " order by key");
+        break;
+      case REPL_TEST_ACID_INSERT_SELECT:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+            "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + " )")
+        .run("insert into " + tableNameOp + " partition (load_date) select * from " + tableName)
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + " )")
+        .run("insert into " + tableNameOp + "_nopart select * from " + tableName + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_IMPORT:
+        String path = "hdfs:///tmp/" + DbName + "/";
+        String exportPath = "'" + path + tableName + "/'";
+        String exportPathNoPart = "'" + path + tableName + "_nopart/'";
+        primary.run("export table " + tableName + " to " + exportPath)
+        .run("import table " + tableNameOp + " from " + exportPath)
+        .run("export table " + tableName + "_nopart to " + exportPathNoPart)
+        .run("import table " + tableNameOp + "_nopart from " + exportPathNoPart);
+        break;
+      case REPL_TEST_ACID_CTAS:
+        primary.run("create table " + tableNameOp + " as select * from " + tableName)
+                .run("create table " + tableNameOp + "_nopart as select * from " + tableName + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_LOADLOCAL:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+              "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+        .run("SHOW TABLES LIKE '" + tableNameOp + "'")
+        .verifyResult(tableNameOp)
+        .run("INSERT OVERWRITE LOCAL DIRECTORY './test.dat' SELECT a.* FROM " + tableName + " a")
+        .run("LOAD DATA LOCAL INPATH './test.dat' OVERWRITE INTO TABLE " + tableNameOp +
+                " PARTITION (load_date='2008-08-15')")
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                      "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+        .run("SHOW TABLES LIKE '" + tableNameOp + "_nopart'")
+        .verifyResult(tableNameOp + "_nopart")
+        .run("LOAD DATA LOCAL INPATH './test.dat' OVERWRITE INTO TABLE " + tableNameOp + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_UNION:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+                .run("SHOW TABLES LIKE '" + tableNameOp + "'")
+                .verifyResult(tableNameOp)
+                .run("insert overwrite table " + tableNameOp + " partition (load_date) select * from " + tableName +
+                    " union all select * from " + tableName)
+                .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+                .run("insert overwrite table " + tableNameOp + "_nopart select * from " + tableName +
+                        "_nopart union all select * from " + tableName + "_nopart");
+        resultArray = new String[]{"1", "2", "3", "4", "5", "1", "2", "3", "4", "5"};
+        break;
+      default:
+        return;
+    }
+    primary.run("select key from " + tableNameOp + " order by key").verifyResults(resultArray);
+    primary.run("select key from " + tableNameOp + "_nopart" + " order by key").verifyResults(resultArray);
+  }
+
+  private String setMMtableProperty(String tableProperty) throws Throwable  {
+    return tableProperty.concat(", 'transactional_properties' = 'insert_only'");
+  }
+
+  private void insertForMerge(String tableName, String tableNameMerge, boolean isMMTable) throws Throwable  {
+    String tableProperty = "'transactional'='true'";
+    if (isMMTable) {
+      tableProperty = setMMtableProperty(tableProperty);
+    }
+    primary.run("use " + primaryDbName)
+        .run("CREATE TABLE " + tableName + "( ID int, TranValue string, last_update_user string) PARTITIONED BY " +
+                "(tran_date string) CLUSTERED BY (ID) into 5 buckets STORED AS ORC TBLPROPERTIES " +
+                " ( "+ tableProperty + " )")
+        .run("SHOW TABLES LIKE '" + tableName + "'")
+        .verifyResult(tableName)
+        .run("CREATE TABLE " + tableNameMerge + " ( ID int, TranValue string, tran_date string) STORED AS ORC ")
+        .run("SHOW TABLES LIKE '" + tableNameMerge + "'")
+        .verifyResult(tableNameMerge)
+        .run("INSERT INTO " + tableName + " PARTITION (tran_date) VALUES (1, 'value_01', 'creation', '20170410')," +
+                " (2, 'value_02', 'creation', '20170410'), (3, 'value_03', 'creation', '20170410'), " +
+                " (4, 'value_04', 'creation', '20170410'), (5, 'value_05', 'creation', '20170413'), " +
+                " (6, 'value_06', 'creation', '20170413'), (7, 'value_07', 'creation', '20170413'),  " +
+                " (8, 'value_08', 'creation', '20170413'), (9, 'value_09', 'creation', '20170413'), " +
+                " (10, 'value_10','creation', '20170413')")
+        .run("select ID from " + tableName + " order by ID")
+        .verifyResults(new String[] {"1", "2", "3", "4", "5", "6", "7", "8", "9", "10"})
+        .run("INSERT INTO " + tableNameMerge + " VALUES (1, 'value_01', '20170410'), " +
+                " (4, NULL, '20170410'), (7, 'value_77777', '20170413'), " +
+                " (8, NULL, '20170413'), (8, 'value_08', '20170415'), " +
+                "(11, 'value_11', '20170415')")
+        .run("select ID from " + tableNameMerge + " order by ID")
+        .verifyResults(new String[] {"1", "4", "7", "8", "8", "11"})
+        .run("MERGE INTO " + tableName + " AS T USING " + tableNameMerge + " AS S ON T.ID = S.ID and" +
+                " T.tran_date = S.tran_date WHEN MATCHED AND (T.TranValue != S.TranValue AND S.TranValue " +
+                " IS NOT NULL) THEN UPDATE SET TranValue = S.TranValue, last_update_user = " +
+                " 'merge_update' WHEN MATCHED AND S.TranValue IS NULL THEN DELETE WHEN NOT MATCHED " +
+                " THEN INSERT VALUES (S.ID, S.TranValue,'merge_insert', S.tran_date)")
+        .run("select last_update_user from " + tableName + " order by last_update_user")
+        .verifyResults(new String[] {"creation", "creation", "creation", "creation", "creation",
+                "creation", "creation", "merge_update", "merge_insert", "merge_insert"});
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
index ff7f9bc..16c124c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
@@ -321,8 +321,7 @@ public class TestReplicationScenariosAcrossInstances {
             "clustered by(key) into 2 buckets stored as orc tblproperties ('transactional'='true')")
         .run("create table table1 (i int, j int)")
         .run("insert into table1 values (1,2)")
-        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     replica.load(replicatedDbName, tuple.dumpLocation)
         .run("use " + replicatedDbName)
@@ -341,8 +340,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("create table table2 (a int, city string) partitioned by (country string)")
         .run("create table table3 (i int, j int)")
         .run("insert into table1 values (1,2)")
-        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     replica.load(replicatedDbName, bootstrapTuple.dumpLocation)
         .run("use " + replicatedDbName)
@@ -467,8 +465,7 @@ public class TestReplicationScenariosAcrossInstances {
                 SOURCE_OF_REPLICATION + "' = '1,2,3')")
         .run("use " + dbTwo)
         .run("create table t1 (i int, j int)")
-        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     /*
       Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM
@@ -527,8 +524,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("use " + dbOne)
         .run("create table t1 (i int, j int) partitioned by (load_date date) "
             + "clustered by(i) into 2 buckets stored as orc tblproperties ('transactional'='true') ")
-        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     String dbTwo = primaryDbName + randomTwo;
     WarehouseInstance.Tuple incrementalTuple = primary
@@ -539,8 +535,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("use " + dbOne)
         .run("create table t2 (a int, b int)")
         .dump("`*`", bootstrapTuple.lastReplicationId,
-            Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-                "'hive.repl.dump.include.acid.tables'='true'"));
+            Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     /*
       Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index f666df1..1e3478d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -249,6 +249,11 @@ public class WarehouseInstance implements Closeable {
     return this;
   }
 
+  WarehouseInstance loadWithoutExplain(String replicatedDbName, String dumpLocation) throws Throwable {
+    run("REPL LOAD " + replicatedDbName + " FROM '" + dumpLocation + "'");
+    return this;
+  }
+
   WarehouseInstance load(String replicatedDbName, String dumpLocation, List<String> withClauseOptions)
           throws Throwable {
     String replLoadCmd = "REPL LOAD " + replicatedDbName + " FROM '" + dumpLocation + "'";

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
index f87a6aa..2ba6d07 100644
--- a/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.thrift.TException;
 
 
@@ -109,6 +110,10 @@ public final class SynchronizedMetaStoreClient {
     return client.fireListenerEvent(rqst);
   }
 
+  public synchronized void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.addWriteNotificationLog(rqst);
+  }
+
   public synchronized void close() {
     client.close();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 19097f5..bf7749d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -139,7 +139,11 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
       if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS)) {
         deletePath = createTargetPath(targetPath, tgtFs);
       }
-      Hive.clearDestForSubDirSrc(conf, targetPath, sourcePath, false);
+      //For acid table incremental replication, just copy the content of staging directory to destination.
+      //No need to clean it.
+      if (work.isNeedCleanTarget()) {
+        Hive.clearDestForSubDirSrc(conf, targetPath, sourcePath, false);
+      }
       // Set isManaged to false as this is not load data operation for which it is needed.
       if (!Hive.moveFile(conf, sourcePath, targetPath, true, false, false)) {
         try {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
index 3a7f1bc..d095de6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
@@ -151,10 +151,11 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
           continue;
         }
         String destFileName = srcFile.getCmPath().getName();
-        Path destFile = new Path(toPath, destFileName);
+        Path destRoot = CopyUtils.getCopyDestination(srcFile, toPath);
+        Path destFile = new Path(destRoot, destFileName);
         if (dstFs.exists(destFile)) {
           String destFileWithSourceName = srcFile.getSourcePath().getName();
-          Path newDestFile = new Path(toPath, destFileWithSourceName);
+          Path newDestFile = new Path(destRoot, destFileWithSourceName);
           boolean result = dstFs.rename(destFile, newDestFile);
           if (!result) {
             throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
index 5bbc25a..c2953c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
@@ -60,8 +62,19 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
           return 0;
         }
       } catch (InvalidTableException e) {
-        LOG.info("Table does not exist so, ignoring the operation as it might be a retry(idempotent) case.");
-        return 0;
+        // In scenarios like import to mm tables, the alloc write id event is generated before create table event.
+        try {
+          Database database = Hive.get().getDatabase(work.getDbName());
+          if (!replicationSpec.allowReplacementInto(database.getParameters())) {
+            // if the event is already replayed, then no need to replay it again.
+            LOG.debug("ReplTxnTask: Event is skipped as it is already replayed. Event Id: " +
+                    replicationSpec.getReplicationState() + "Event Type: " + work.getOperationType());
+            return 0;
+          }
+        } catch (HiveException e1) {
+          LOG.error("Get database failed with exception " + e1.getMessage());
+          return 1;
+        }
       } catch (HiveException e) {
         LOG.error("Get table failed with exception " + e.getMessage());
         return 1;
@@ -85,10 +98,16 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
         }
         return 0;
       case REPL_COMMIT_TXN:
-        for (long txnId : work.getTxnIds()) {
-          txnManager.replCommitTxn(replPolicy, txnId);
-          LOG.info("Replayed CommitTxn Event for policy " + replPolicy + " with srcTxn " + txnId);
-        }
+        // Currently only one commit txn per event is supported.
+        assert (work.getTxnIds().size() == 1);
+
+        long txnId = work.getTxnIds().get(0);
+        CommitTxnRequest commitTxnRequest = new CommitTxnRequest(txnId);
+        commitTxnRequest.setReplPolicy(work.getReplPolicy());
+        commitTxnRequest.setWriteEventInfos(work.getWriteEventInfos());
+        txnManager.replCommitTxn(commitTxnRequest);
+        LOG.info("Replayed CommitTxn Event for replPolicy: " + replPolicy + " with srcTxn: " + txnId +
+                "WriteEventInfos: " + work.getWriteEventInfos());
         return 0;
       case REPL_ALLOC_WRITE_ID:
         assert work.getTxnToWriteIdList() != null;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
index e48657c..82ecad1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
@@ -199,7 +199,9 @@ public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
         cmRoot,
         getHive(),
         conf,
-        getNewEventOnlyReplicationSpec(ev.getEventId())
+        getNewEventOnlyReplicationSpec(ev.getEventId()),
+        work.dbNameOrPattern,
+        work.tableNameOrPattern
     );
     EventHandler eventHandler = EventHandlerFactory.handlerFor(ev);
     eventHandler.handle(context);


[07/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
new file mode 100644
index 0000000..5758820
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
@@ -0,0 +1,949 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteNotificationLogRequest implements org.apache.thrift.TBase<WriteNotificationLogRequest, WriteNotificationLogRequest._Fields>, java.io.Serializable, Cloneable, Comparable<WriteNotificationLogRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteNotificationLogRequest");
+
+  private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_FIELD_DESC = new org.apache.thrift.protocol.TField("db", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField FILE_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("fileInfo", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField PARTITION_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionVals", org.apache.thrift.protocol.TType.LIST, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteNotificationLogRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteNotificationLogRequestTupleSchemeFactory());
+  }
+
+  private long txnId; // required
+  private long writeId; // required
+  private String db; // required
+  private String table; // required
+  private InsertEventRequestData fileInfo; // required
+  private List<String> partitionVals; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TXN_ID((short)1, "txnId"),
+    WRITE_ID((short)2, "writeId"),
+    DB((short)3, "db"),
+    TABLE((short)4, "table"),
+    FILE_INFO((short)5, "fileInfo"),
+    PARTITION_VALS((short)6, "partitionVals");
+
+    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_ID
+          return TXN_ID;
+        case 2: // WRITE_ID
+          return WRITE_ID;
+        case 3: // DB
+          return DB;
+        case 4: // TABLE
+          return TABLE;
+        case 5: // FILE_INFO
+          return FILE_INFO;
+        case 6: // PARTITION_VALS
+          return PARTITION_VALS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TXNID_ISSET_ID = 0;
+  private static final int __WRITEID_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PARTITION_VALS};
+  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_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("writeId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.DB, new org.apache.thrift.meta_data.FieldMetaData("db", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FILE_INFO, new org.apache.thrift.meta_data.FieldMetaData("fileInfo", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, InsertEventRequestData.class)));
+    tmpMap.put(_Fields.PARTITION_VALS, new org.apache.thrift.meta_data.FieldMetaData("partitionVals", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteNotificationLogRequest.class, metaDataMap);
+  }
+
+  public WriteNotificationLogRequest() {
+  }
+
+  public WriteNotificationLogRequest(
+    long txnId,
+    long writeId,
+    String db,
+    String table,
+    InsertEventRequestData fileInfo)
+  {
+    this();
+    this.txnId = txnId;
+    setTxnIdIsSet(true);
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+    this.db = db;
+    this.table = table;
+    this.fileInfo = fileInfo;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteNotificationLogRequest(WriteNotificationLogRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.txnId = other.txnId;
+    this.writeId = other.writeId;
+    if (other.isSetDb()) {
+      this.db = other.db;
+    }
+    if (other.isSetTable()) {
+      this.table = other.table;
+    }
+    if (other.isSetFileInfo()) {
+      this.fileInfo = new InsertEventRequestData(other.fileInfo);
+    }
+    if (other.isSetPartitionVals()) {
+      List<String> __this__partitionVals = new ArrayList<String>(other.partitionVals);
+      this.partitionVals = __this__partitionVals;
+    }
+  }
+
+  public WriteNotificationLogRequest deepCopy() {
+    return new WriteNotificationLogRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    setTxnIdIsSet(false);
+    this.txnId = 0;
+    setWriteIdIsSet(false);
+    this.writeId = 0;
+    this.db = null;
+    this.table = null;
+    this.fileInfo = null;
+    this.partitionVals = null;
+  }
+
+  public long getTxnId() {
+    return this.txnId;
+  }
+
+  public void setTxnId(long txnId) {
+    this.txnId = txnId;
+    setTxnIdIsSet(true);
+  }
+
+  public void unsetTxnId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID);
+  }
+
+  /** Returns true if field txnId is set (has been assigned a value) and false otherwise */
+  public boolean isSetTxnId() {
+    return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID);
+  }
+
+  public void setTxnIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value);
+  }
+
+  public long getWriteId() {
+    return this.writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+  }
+
+  public void unsetWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field writeId is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  public void setWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEID_ISSET_ID, value);
+  }
+
+  public String getDb() {
+    return this.db;
+  }
+
+  public void setDb(String db) {
+    this.db = db;
+  }
+
+  public void unsetDb() {
+    this.db = null;
+  }
+
+  /** Returns true if field db is set (has been assigned a value) and false otherwise */
+  public boolean isSetDb() {
+    return this.db != null;
+  }
+
+  public void setDbIsSet(boolean value) {
+    if (!value) {
+      this.db = null;
+    }
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public void unsetTable() {
+    this.table = null;
+  }
+
+  /** Returns true if field table is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable() {
+    return this.table != null;
+  }
+
+  public void setTableIsSet(boolean value) {
+    if (!value) {
+      this.table = null;
+    }
+  }
+
+  public InsertEventRequestData getFileInfo() {
+    return this.fileInfo;
+  }
+
+  public void setFileInfo(InsertEventRequestData fileInfo) {
+    this.fileInfo = fileInfo;
+  }
+
+  public void unsetFileInfo() {
+    this.fileInfo = null;
+  }
+
+  /** Returns true if field fileInfo is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileInfo() {
+    return this.fileInfo != null;
+  }
+
+  public void setFileInfoIsSet(boolean value) {
+    if (!value) {
+      this.fileInfo = null;
+    }
+  }
+
+  public int getPartitionValsSize() {
+    return (this.partitionVals == null) ? 0 : this.partitionVals.size();
+  }
+
+  public java.util.Iterator<String> getPartitionValsIterator() {
+    return (this.partitionVals == null) ? null : this.partitionVals.iterator();
+  }
+
+  public void addToPartitionVals(String elem) {
+    if (this.partitionVals == null) {
+      this.partitionVals = new ArrayList<String>();
+    }
+    this.partitionVals.add(elem);
+  }
+
+  public List<String> getPartitionVals() {
+    return this.partitionVals;
+  }
+
+  public void setPartitionVals(List<String> partitionVals) {
+    this.partitionVals = partitionVals;
+  }
+
+  public void unsetPartitionVals() {
+    this.partitionVals = null;
+  }
+
+  /** Returns true if field partitionVals is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitionVals() {
+    return this.partitionVals != null;
+  }
+
+  public void setPartitionValsIsSet(boolean value) {
+    if (!value) {
+      this.partitionVals = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TXN_ID:
+      if (value == null) {
+        unsetTxnId();
+      } else {
+        setTxnId((Long)value);
+      }
+      break;
+
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
+    case DB:
+      if (value == null) {
+        unsetDb();
+      } else {
+        setDb((String)value);
+      }
+      break;
+
+    case TABLE:
+      if (value == null) {
+        unsetTable();
+      } else {
+        setTable((String)value);
+      }
+      break;
+
+    case FILE_INFO:
+      if (value == null) {
+        unsetFileInfo();
+      } else {
+        setFileInfo((InsertEventRequestData)value);
+      }
+      break;
+
+    case PARTITION_VALS:
+      if (value == null) {
+        unsetPartitionVals();
+      } else {
+        setPartitionVals((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TXN_ID:
+      return getTxnId();
+
+    case WRITE_ID:
+      return getWriteId();
+
+    case DB:
+      return getDb();
+
+    case TABLE:
+      return getTable();
+
+    case FILE_INFO:
+      return getFileInfo();
+
+    case PARTITION_VALS:
+      return getPartitionVals();
+
+    }
+    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_ID:
+      return isSetTxnId();
+    case WRITE_ID:
+      return isSetWriteId();
+    case DB:
+      return isSetDb();
+    case TABLE:
+      return isSetTable();
+    case FILE_INFO:
+      return isSetFileInfo();
+    case PARTITION_VALS:
+      return isSetPartitionVals();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteNotificationLogRequest)
+      return this.equals((WriteNotificationLogRequest)that);
+    return false;
+  }
+
+  public boolean equals(WriteNotificationLogRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_txnId = true;
+    boolean that_present_txnId = true;
+    if (this_present_txnId || that_present_txnId) {
+      if (!(this_present_txnId && that_present_txnId))
+        return false;
+      if (this.txnId != that.txnId)
+        return false;
+    }
+
+    boolean this_present_writeId = true;
+    boolean that_present_writeId = true;
+    if (this_present_writeId || that_present_writeId) {
+      if (!(this_present_writeId && that_present_writeId))
+        return false;
+      if (this.writeId != that.writeId)
+        return false;
+    }
+
+    boolean this_present_db = true && this.isSetDb();
+    boolean that_present_db = true && that.isSetDb();
+    if (this_present_db || that_present_db) {
+      if (!(this_present_db && that_present_db))
+        return false;
+      if (!this.db.equals(that.db))
+        return false;
+    }
+
+    boolean this_present_table = true && this.isSetTable();
+    boolean that_present_table = true && that.isSetTable();
+    if (this_present_table || that_present_table) {
+      if (!(this_present_table && that_present_table))
+        return false;
+      if (!this.table.equals(that.table))
+        return false;
+    }
+
+    boolean this_present_fileInfo = true && this.isSetFileInfo();
+    boolean that_present_fileInfo = true && that.isSetFileInfo();
+    if (this_present_fileInfo || that_present_fileInfo) {
+      if (!(this_present_fileInfo && that_present_fileInfo))
+        return false;
+      if (!this.fileInfo.equals(that.fileInfo))
+        return false;
+    }
+
+    boolean this_present_partitionVals = true && this.isSetPartitionVals();
+    boolean that_present_partitionVals = true && that.isSetPartitionVals();
+    if (this_present_partitionVals || that_present_partitionVals) {
+      if (!(this_present_partitionVals && that_present_partitionVals))
+        return false;
+      if (!this.partitionVals.equals(that.partitionVals))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_txnId = true;
+    list.add(present_txnId);
+    if (present_txnId)
+      list.add(txnId);
+
+    boolean present_writeId = true;
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
+    boolean present_db = true && (isSetDb());
+    list.add(present_db);
+    if (present_db)
+      list.add(db);
+
+    boolean present_table = true && (isSetTable());
+    list.add(present_table);
+    if (present_table)
+      list.add(table);
+
+    boolean present_fileInfo = true && (isSetFileInfo());
+    list.add(present_fileInfo);
+    if (present_fileInfo)
+      list.add(fileInfo);
+
+    boolean present_partitionVals = true && (isSetPartitionVals());
+    list.add(present_partitionVals);
+    if (present_partitionVals)
+      list.add(partitionVals);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteNotificationLogRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTxnId()).compareTo(other.isSetTxnId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTxnId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnId, other.txnId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetWriteId()).compareTo(other.isSetWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeId, other.writeId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDb()).compareTo(other.isSetDb());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDb()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db, other.db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFileInfo()).compareTo(other.isSetFileInfo());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileInfo()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileInfo, other.fileInfo);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartitionVals()).compareTo(other.isSetPartitionVals());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitionVals()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionVals, other.partitionVals);
+      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("WriteNotificationLogRequest(");
+    boolean first = true;
+
+    sb.append("txnId:");
+    sb.append(this.txnId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("writeId:");
+    sb.append(this.writeId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("db:");
+    if (this.db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table:");
+    if (this.table == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fileInfo:");
+    if (this.fileInfo == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileInfo);
+    }
+    first = false;
+    if (isSetPartitionVals()) {
+      if (!first) sb.append(", ");
+      sb.append("partitionVals:");
+      if (this.partitionVals == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partitionVals);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetTxnId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'txnId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'writeId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetDb()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'db' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'table' is unset! Struct:" + toString());
+    }
+
+    if (!isSetFileInfo()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileInfo' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (fileInfo != null) {
+      fileInfo.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WriteNotificationLogRequestStandardSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogRequestStandardScheme getScheme() {
+      return new WriteNotificationLogRequestStandardScheme();
+    }
+  }
+
+  private static class WriteNotificationLogRequestStandardScheme extends StandardScheme<WriteNotificationLogRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLogRequest 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_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.txnId = iprot.readI64();
+              struct.setTxnIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.writeId = iprot.readI64();
+              struct.setWriteIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.db = iprot.readString();
+              struct.setDbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table = iprot.readString();
+              struct.setTableIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // FILE_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.fileInfo = new InsertEventRequestData();
+              struct.fileInfo.read(iprot);
+              struct.setFileInfoIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // PARTITION_VALS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list764.size);
+                String _elem765;
+                for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                {
+                  _elem765 = iprot.readString();
+                  struct.partitionVals.add(_elem765);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPartitionValsIsSet(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, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
+      oprot.writeI64(struct.txnId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.writeId);
+      oprot.writeFieldEnd();
+      if (struct.db != null) {
+        oprot.writeFieldBegin(DB_FIELD_DESC);
+        oprot.writeString(struct.db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table != null) {
+        oprot.writeFieldBegin(TABLE_FIELD_DESC);
+        oprot.writeString(struct.table);
+        oprot.writeFieldEnd();
+      }
+      if (struct.fileInfo != null) {
+        oprot.writeFieldBegin(FILE_INFO_FIELD_DESC);
+        struct.fileInfo.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.partitionVals != null) {
+        if (struct.isSetPartitionVals()) {
+          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 _iter767 : struct.partitionVals)
+            {
+              oprot.writeString(_iter767);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteNotificationLogRequestTupleSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogRequestTupleScheme getScheme() {
+      return new WriteNotificationLogRequestTupleScheme();
+    }
+  }
+
+  private static class WriteNotificationLogRequestTupleScheme extends TupleScheme<WriteNotificationLogRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.txnId);
+      oprot.writeI64(struct.writeId);
+      oprot.writeString(struct.db);
+      oprot.writeString(struct.table);
+      struct.fileInfo.write(oprot);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartitionVals()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetPartitionVals()) {
+        {
+          oprot.writeI32(struct.partitionVals.size());
+          for (String _iter768 : struct.partitionVals)
+          {
+            oprot.writeString(_iter768);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.txnId = iprot.readI64();
+      struct.setTxnIdIsSet(true);
+      struct.writeId = iprot.readI64();
+      struct.setWriteIdIsSet(true);
+      struct.db = iprot.readString();
+      struct.setDbIsSet(true);
+      struct.table = iprot.readString();
+      struct.setTableIsSet(true);
+      struct.fileInfo = new InsertEventRequestData();
+      struct.fileInfo.read(iprot);
+      struct.setFileInfoIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list769.size);
+          String _elem770;
+          for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+          {
+            _elem770 = iprot.readString();
+            struct.partitionVals.add(_elem770);
+          }
+        }
+        struct.setPartitionValsIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
new file mode 100644
index 0000000..fab4da2
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
@@ -0,0 +1,283 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteNotificationLogResponse implements org.apache.thrift.TBase<WriteNotificationLogResponse, WriteNotificationLogResponse._Fields>, java.io.Serializable, Cloneable, Comparable<WriteNotificationLogResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteNotificationLogResponse");
+
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteNotificationLogResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteNotificationLogResponseTupleSchemeFactory());
+  }
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+    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) {
+        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;
+    }
+  }
+  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);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteNotificationLogResponse.class, metaDataMap);
+  }
+
+  public WriteNotificationLogResponse() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteNotificationLogResponse(WriteNotificationLogResponse other) {
+  }
+
+  public WriteNotificationLogResponse deepCopy() {
+    return new WriteNotificationLogResponse(this);
+  }
+
+  @Override
+  public void clear() {
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    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) {
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteNotificationLogResponse)
+      return this.equals((WriteNotificationLogResponse)that);
+    return false;
+  }
+
+  public boolean equals(WriteNotificationLogResponse that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteNotificationLogResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    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("WriteNotificationLogResponse(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // 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 WriteNotificationLogResponseStandardSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogResponseStandardScheme getScheme() {
+      return new WriteNotificationLogResponseStandardScheme();
+    }
+  }
+
+  private static class WriteNotificationLogResponseStandardScheme extends StandardScheme<WriteNotificationLogResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLogResponse 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) {
+          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, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteNotificationLogResponseTupleSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogResponseTupleScheme getScheme() {
+      return new WriteNotificationLogResponseTupleScheme();
+    }
+  }
+
+  private static class WriteNotificationLogResponseTupleScheme extends TupleScheme<WriteNotificationLogResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+    }
+  }
+
+}
+


[02/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
index f7018c2..ac1d3c8 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
@@ -59,6 +59,7 @@ public class ReplChangeManager {
   static final String REMAIN_IN_TRASH_TAG = "user.remain-in-trash";
   private static final String URI_FRAGMENT_SEPARATOR = "#";
   public static final String SOURCE_OF_REPLICATION = "repl.source.for";
+  private static final String TXN_WRITE_EVENT_FILE_SEPARATOR = "]";
 
   public enum RecycleType {
     MOVE,
@@ -472,7 +473,6 @@ public class ReplChangeManager {
   }
 
   public static boolean isSourceOfReplication(Database db) {
-    // Can not judge, so assuming replication is not enabled.
     assert (db != null);
     String replPolicyIds = getReplPolicyIdString(db);
     return  !StringUtils.isEmpty(replPolicyIds);
@@ -490,4 +490,12 @@ public class ReplChangeManager {
     }
     return null;
   }
+
+  public static String joinWithSeparator(Iterable<?> strings) {
+    return org.apache.hadoop.util.StringUtils.join(TXN_WRITE_EVENT_FILE_SEPARATOR, strings);
+  }
+
+  public static String[] getListFromSeparatedString(String commaSeparatedString) {
+    return commaSeparatedString.split("\\s*" + TXN_WRITE_EVENT_FILE_SEPARATOR + "\\s*");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index a526019..8ff056f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -111,6 +111,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -2414,6 +2415,17 @@ public class CachedStore implements RawStore, Configurable {
     return sharedCache.getUpdateCount();
   }
 
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    rawStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return rawStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
   static boolean isNotInBlackList(String catName, String dbName, String tblName) {
     String str = TableName.getQualified(catName, dbName, tblName);
     for (Pattern pattern : blacklistPatterns) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
new file mode 100644
index 0000000..001179a
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
@@ -0,0 +1,91 @@
+/*
+ * 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.metastore.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
+
+import java.util.List;
+
+/**
+ * AcidWriteEvent
+ * Event generated for acid write operations
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AcidWriteEvent extends ListenerEvent {
+  private final WriteNotificationLogRequest writeNotificationLogRequest;
+  private final String partition;
+  private final Table tableObj;
+  private final Partition partitionObj;
+
+  public AcidWriteEvent(String partition, Table tableObj, Partition partitionObj,
+                        WriteNotificationLogRequest writeNotificationLogRequest) {
+    super(true, null);
+    this.writeNotificationLogRequest = writeNotificationLogRequest;
+    this.partition = partition;
+    this.tableObj = tableObj;
+    this.partitionObj = partitionObj;
+  }
+
+  public Long getTxnId() {
+    return writeNotificationLogRequest.getTxnId();
+  }
+
+  public List<String> getFiles() {
+    return writeNotificationLogRequest.getFileInfo().getFilesAdded();
+  }
+
+  public List<String> getChecksums() {
+    return writeNotificationLogRequest.getFileInfo().getFilesAddedChecksum();
+  }
+
+  public String getDatabase() {
+    return StringUtils.normalizeIdentifier(writeNotificationLogRequest.getDb());
+  }
+
+  public String getTable() {
+    return StringUtils.normalizeIdentifier(writeNotificationLogRequest.getTable());
+  }
+
+  public String getPartition() {
+    return partition; //Don't normalize partition value, as its case sensitive.
+  }
+
+  public Long getWriteId() {
+    return writeNotificationLogRequest.getWriteId();
+  }
+
+  public Table getTableObj() {
+    return tableObj;
+  }
+
+  public Partition getPartitionObj() {
+    return partitionObj;
+  }
+
+  public List<String> getSubDirs() {
+    return writeNotificationLogRequest.getFileInfo().getSubDirectoryList();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
new file mode 100644
index 0000000..e2c9ccf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
@@ -0,0 +1,50 @@
+/* * 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.metastore.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import java.util.List;
+
+/**
+ * HCat message sent when an ACID write is done.
+ */
+public abstract class AcidWriteMessage extends EventMessage {
+
+  protected AcidWriteMessage() {
+    super(EventType.ACID_WRITE);
+  }
+
+  public abstract Long getTxnId();
+
+  public abstract String getTable();
+
+  public abstract Long getWriteId();
+
+  public abstract String getPartition();
+
+  public abstract List<String> getFiles();
+
+  public abstract Table getTableObj() throws Exception;
+
+  public abstract Partition getPartitionObj() throws Exception;
+
+  public abstract String getTableObjStr();
+
+  public abstract String getPartitionObjStr();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
index 49004f2..9733039 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
@@ -17,6 +17,12 @@
 
 package org.apache.hadoop.hive.metastore.messaging;
 
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+
+import java.util.List;
+
 /**
  * HCat message sent when an commit transaction is done.
  */
@@ -33,4 +39,21 @@ public abstract class CommitTxnMessage extends EventMessage {
    */
   public abstract Long getTxnId();
 
+  public abstract List<Long> getWriteIds();
+
+  public abstract List<String> getDatabases();
+
+  public abstract List<String> getTables();
+
+  public abstract List<String> getPartitions();
+
+  public abstract Table getTableObj(int idx) throws Exception;
+
+  public abstract Partition getPartitionObj(int idx) throws Exception;
+
+  public abstract String getFiles(int idx);
+
+  public abstract List<String> getFilesList();
+
+  public abstract void addWriteEventInfo(List<WriteEventInfo> writeEventInfoList);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
index 969dd7b..f24b419 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -60,7 +60,8 @@ public abstract class EventMessage {
     COMMIT_TXN(MessageFactory.COMMIT_TXN_EVENT),
     ABORT_TXN(MessageFactory.ABORT_TXN_EVENT),
     ALLOC_WRITE_ID(MessageFactory.ALLOC_WRITE_ID_EVENT),
-    ALTER_CATALOG(MessageFactory.ALTER_CATALOG_EVENT);
+    ALTER_CATALOG(MessageFactory.ALTER_CATALOG_EVENT),
+    ACID_WRITE(MessageFactory.ACID_WRITE_EVENT);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
index ca33579..b701d84 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
@@ -70,6 +70,10 @@ public abstract class MessageDeserializer {
       return getCommitTxnMessage(messageBody);
     case ABORT_TXN:
       return getAbortTxnMessage(messageBody);
+    case ALLOC_WRITE_ID:
+      return getAllocWriteIdMessage(messageBody);
+    case ACID_WRITE:
+      return getAcidWriteMessage(messageBody);
     default:
       throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString);
     }
@@ -186,6 +190,11 @@ public abstract class MessageDeserializer {
    */
   public abstract AllocWriteIdMessage getAllocWriteIdMessage(String messageBody);
 
+  /*
+   * Method to de-serialize AcidWriteMessage instance.
+   */
+  public abstract AcidWriteMessage getAcidWriteMessage(String messageBody);
+
   // Protection against construction.
   protected MessageDeserializer() {}
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index e0629ea..d529147 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 
 import java.util.Iterator;
@@ -74,6 +75,7 @@ public abstract class MessageFactory {
   public static final String ABORT_TXN_EVENT = "ABORT_TXN";
   public static final String ALLOC_WRITE_ID_EVENT = "ALLOC_WRITE_ID_EVENT";
   public static final String ALTER_CATALOG_EVENT = "ALTER_CATALOG";
+  public static final String ACID_WRITE_EVENT = "ACID_WRITE_EVENT";
 
   private static MessageFactory instance = null;
 
@@ -326,4 +328,14 @@ public abstract class MessageFactory {
   public abstract DropCatalogMessage buildDropCatalogMessage(Catalog catalog);
 
   public abstract AlterCatalogMessage buildAlterCatalogMessage(Catalog oldCat, Catalog newCat);
+
+  /**
+   * Factory method for building acid write message
+   *
+   *
+   * @param acidWriteEvent information related to the acid write operation
+   * @param files files added by this write operation
+   * @return instance of AcidWriteMessage
+   */
+  public abstract AcidWriteMessage buildAcidWriteMessage(AcidWriteEvent acidWriteEvent, Iterator<String> files);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
new file mode 100644
index 0000000..515a2cb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
@@ -0,0 +1,150 @@
+/*
+ * 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.metastore.messaging.json;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * JSON implementation of AcidWriteMessage
+ */
+public class JSONAcidWriteMessage extends AcidWriteMessage {
+
+  @JsonProperty
+  private Long txnid, writeId, timestamp;
+
+  @JsonProperty
+  private String server, servicePrincipal, database, table, partition, tableObjJson, partitionObjJson;
+
+  @JsonProperty
+  private List<String> files;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAcidWriteMessage() {
+  }
+
+  public JSONAcidWriteMessage(String server, String servicePrincipal, Long timestamp, AcidWriteEvent acidWriteEvent,
+                              Iterator<String> files) {
+    this.timestamp = timestamp;
+    this.txnid = acidWriteEvent.getTxnId();
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.database = acidWriteEvent.getDatabase();
+    this.table = acidWriteEvent.getTable();
+    this.writeId = acidWriteEvent.getWriteId();
+    this.partition = acidWriteEvent.getPartition();
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(acidWriteEvent.getTableObj());
+      if (acidWriteEvent.getPartitionObj() != null) {
+        this.partitionObjJson = JSONMessageFactory.createPartitionObjJson(acidWriteEvent.getPartitionObj());
+      } else {
+        this.partitionObjJson = null;
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize JSONAcidWriteMessage : ", e);
+    }
+    this.files = Lists.newArrayList(files);
+  }
+
+  @Override
+  public Long getTxnId() {
+    return txnid;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return database;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public Long getWriteId() {
+    return writeId;
+  }
+
+  @Override
+  public String getPartition() {
+    return partition;
+  }
+
+  @Override
+  public List<String> getFiles() {
+    return files;
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (tableObjJson == null) ? null : (Table) JSONMessageFactory.getTObj(tableObjJson, Table.class);
+  }
+
+  @Override
+  public Partition getPartitionObj() throws Exception {
+    return ((partitionObjJson == null) ? null :
+            (Partition) JSONMessageFactory.getTObj(partitionObjJson, Partition.class));
+  }
+
+  @Override
+  public String getTableObjStr() {
+    return tableObjJson;
+  }
+
+  @Override
+  public String getPartitionObjStr() {
+    return partitionObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
index 595a3d1..6082b8e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
@@ -18,9 +18,15 @@
  */
 
 package org.apache.hadoop.hive.metastore.messaging.json;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.codehaus.jackson.annotate.JsonProperty;
 
+import java.util.List;
+
 /**
  * JSON implementation of CommitTxnMessage
  */
@@ -38,6 +44,12 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
   @JsonProperty
   private String servicePrincipal;
 
+  @JsonProperty
+  private List<Long> writeIds;
+
+  @JsonProperty
+  private List<String> databases, tables, partitions, tableObjs, partitionObjs, files;
+
   /**
    * Default constructor, needed for Jackson.
    */
@@ -49,6 +61,13 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
     this.txnid = txnid;
     this.server = server;
     this.servicePrincipal = servicePrincipal;
+    this.databases = null;
+    this.tables = null;
+    this.writeIds = null;
+    this.partitions = null;
+    this.tableObjs = null;
+    this.partitionObjs = null;
+    this.files = null;
   }
 
   @Override
@@ -77,6 +96,82 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
   }
 
   @Override
+  public List<Long> getWriteIds() {
+    return writeIds;
+  }
+
+  @Override
+  public List<String> getDatabases() {
+    return databases;
+  }
+
+  @Override
+  public List<String> getTables() {
+    return tables;
+  }
+
+  @Override
+  public List<String> getPartitions() {
+    return partitions;
+  }
+
+  @Override
+  public Table getTableObj(int idx) throws Exception {
+    return tableObjs == null ? null :  (Table) JSONMessageFactory.getTObj(tableObjs.get(idx), Table.class);
+  }
+
+  @Override
+  public Partition getPartitionObj(int idx) throws Exception {
+    return (partitionObjs == null ? null : (partitionObjs.get(idx) == null ? null :
+            (Partition)JSONMessageFactory.getTObj(partitionObjs.get(idx), Partition.class)));
+  }
+
+  @Override
+  public String getFiles(int idx) {
+    return files == null ? null : files.get(idx);
+  }
+
+  @Override
+  public List<String> getFilesList() {
+    return files;
+  }
+
+  @Override
+  public void addWriteEventInfo(List<WriteEventInfo> writeEventInfoList) {
+    if (this.databases == null) {
+      this.databases = Lists.newArrayList();
+    }
+    if (this.tables == null) {
+      this.tables = Lists.newArrayList();
+    }
+    if (this.writeIds == null) {
+      this.writeIds = Lists.newArrayList();
+    }
+    if (this.tableObjs == null) {
+      this.tableObjs = Lists.newArrayList();
+    }
+    if (this.partitions == null) {
+      this.partitions = Lists.newArrayList();
+    }
+    if (this.partitionObjs == null) {
+      this.partitionObjs = Lists.newArrayList();
+    }
+    if (this.files == null) {
+      this.files = Lists.newArrayList();
+    }
+
+    for (WriteEventInfo writeEventInfo : writeEventInfoList) {
+      this.databases.add(writeEventInfo.getDatabase());
+      this.tables.add(writeEventInfo.getTable());
+      this.writeIds.add(writeEventInfo.getWriteId());
+      this.partitions.add(writeEventInfo.getPartition());
+      this.tableObjs.add(writeEventInfo.getTableObj());
+      this.partitionObjs.add(writeEventInfo.getPartitionObj());
+      this.files.add(writeEventInfo.getFiles());
+    }
+  }
+
+  @Override
   public String toString() {
     try {
       return JSONMessageDeserializer.mapper.writeValueAsString(this);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
index f54e24d..be6b751 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.codehaus.jackson.map.DeserializationConfig;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
@@ -259,4 +260,12 @@ public class JSONMessageDeserializer extends MessageDeserializer {
       throw new IllegalArgumentException("Could not construct AllocWriteIdMessage", e);
     }
   }
+
+  public AcidWriteMessage getAcidWriteMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAcidWriteMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AcidWriteMessage", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index d64c3ff..07f51f0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
@@ -66,6 +67,7 @@ import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
@@ -230,11 +232,17 @@ public class JSONMessageFactory extends MessageFactory {
     return new JSONAbortTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnId, now());
   }
 
+  @Override
   public AllocWriteIdMessage buildAllocWriteIdMessage(List<TxnToWriteId> txnToWriteIdList,
                                                       String dbName, String tableName) {
     return new JSONAllocWriteIdMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnToWriteIdList, dbName, tableName, now());
   }
 
+  @Override
+  public AcidWriteMessage buildAcidWriteMessage(AcidWriteEvent acidWriteEvent, Iterator<String> files) {
+    return new JSONAcidWriteMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, now(), acidWriteEvent, files);
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
new file mode 100644
index 0000000..f5ca386
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
@@ -0,0 +1,123 @@
+/*
+ * 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.metastore.model;
+
+/**
+ * MTxnWriteNotificationLog
+ * DN table for ACID write events.
+ */
+public class MTxnWriteNotificationLog {
+  private long txnId;
+  private long writeId;
+  private int eventTime;
+  private String database;
+  private String table;
+  private String partition;
+  private String tableObject;
+  private String partObject;
+  private String files;
+
+  public MTxnWriteNotificationLog() {
+  }
+
+  public MTxnWriteNotificationLog(long txnId, long writeId, int eventTime, String database, String table,
+                               String partition, String tableObject, String partObject, String files) {
+    this.txnId = txnId;
+    this.writeId = writeId;
+    this.eventTime = eventTime;
+    this.database = database;
+    this.table = table;
+    this.partition = partition;
+    this.tableObject = tableObject;
+    this.partObject = partObject;
+    this.files = files;
+  }
+
+  public long getTxnId() {
+    return txnId;
+  }
+
+  public void setTxnId(long txnId) {
+    this.txnId = txnId;
+  }
+
+  public long getWriteId() {
+    return writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+  }
+
+  public int getEventTime() {
+    return eventTime;
+  }
+
+  public void setEventTime(int eventTime) {
+    this.eventTime = eventTime;
+  }
+
+  public String getDatabase() {
+    return database;
+  }
+
+  public void setDatabase(String database) {
+    this.database = database;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public String getPartition() {
+    return partition;
+  }
+
+  public void setPartition(String partition) {
+    this.partition = partition;
+  }
+
+  public String getTableObject() {
+    return tableObject;
+  }
+
+  public void setTableObject(String tableObject) {
+    this.tableObject = tableObject;
+  }
+
+  public String getPartObject() {
+    return partObject;
+  }
+
+  public void setPartObject(String partObject) {
+    this.partObject = partObject;
+  }
+
+  public String getFiles() {
+    return files;
+  }
+
+  public void setFiles(String files) {
+    this.files = files;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
index b23a6d7..d0ac7db 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
@@ -175,4 +175,13 @@ public final class SQLGenerator {
     return dbProduct;
   }
 
+  // This is required for SQL executed directly. If the SQL has double quotes then some dbs tend to
+  // remove the escape characters and store the variable without double quote.
+  public String addEscapeCharacters(String s) {
+    if (dbProduct ==  DatabaseProduct.MYSQL) {
+      return s.replaceAll("\\\\", "\\\\\\\\");
+    }
+    return s;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 50bfca3..f8c2ca2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -245,6 +245,34 @@ public final class TxnDbUtil {
       stmt.execute("INSERT INTO \"APP\".\"NOTIFICATION_SEQUENCE\" (\"NNI_ID\", \"NEXT_EVENT_ID\")" +
               " SELECT * FROM (VALUES (1,1)) tmp_table WHERE NOT EXISTS ( SELECT " +
               "\"NEXT_EVENT_ID\" FROM \"APP\".\"NOTIFICATION_SEQUENCE\")");
+
+      try {
+        stmt.execute("CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (" +
+                "WNL_ID bigint NOT NULL," +
+                "WNL_TXNID bigint NOT NULL," +
+                "WNL_WRITEID bigint NOT NULL," +
+                "WNL_DATABASE varchar(128) NOT NULL," +
+                "WNL_TABLE varchar(128) NOT NULL," +
+                "WNL_PARTITION varchar(1024) NOT NULL," +
+                "WNL_TABLE_OBJ clob NOT NULL," +
+                "WNL_PARTITION_OBJ clob," +
+                "WNL_FILES clob," +
+                "WNL_EVENT_TIME integer NOT NULL," +
+                "PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION))"
+        );
+      } catch (SQLException e) {
+        if (e.getMessage() != null && e.getMessage().contains("already exists")) {
+          LOG.info("TXN_WRITE_NOTIFICATION_LOG table already exist, ignoring");
+        } else {
+          throw e;
+        }
+      }
+
+      stmt.execute("INSERT INTO \"APP\".\"SEQUENCE_TABLE\" (\"SEQUENCE_NAME\", \"NEXT_VAL\") " +
+              "SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', " +
+              "1)) tmp_table WHERE NOT EXISTS ( SELECT \"NEXT_VAL\" FROM \"APP\"" +
+              ".\"SEQUENCE_TABLE\" WHERE \"SEQUENCE_NAME\" = 'org.apache.hadoop.hive.metastore" +
+              ".model.MTxnWriteNotificationLog')");
     } catch (SQLException e) {
       try {
         conn.rollback();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 361ede5..3785f89 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
@@ -698,6 +699,38 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   @Override
   @RetrySemantics.Idempotent
+  public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException {
+    try {
+      Connection dbConn = null;
+      Statement stmt = null;
+      try {
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        stmt = dbConn.createStatement();
+        List<Long> targetTxnIds = getTargetTxnIdList(replPolicy, Collections.singletonList(sourceTxnId), stmt);
+        if (targetTxnIds.isEmpty()) {
+          LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy);
+          return -1;
+        }
+        assert (targetTxnIds.size() == 1);
+        return targetTxnIds.get(0);
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(dbConn, e, "getTargetTxnId(" + replPolicy + sourceTxnId + ")");
+        throw new MetaException("Unable to get target transaction id "
+                + StringUtils.stringifyException(e));
+      } finally {
+        close(null, stmt, dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      return getTargetTxnId(replPolicy, sourceTxnId);
+    }
+  }
+
+  @Override
+  @RetrySemantics.Idempotent
   public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException {
     long txnid = rqst.getTxnid();
     long sourceTxnId = -1;
@@ -892,10 +925,17 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           shouldNeverHappen(txnid);
           //dbConn is rolled back in finally{}
         }
-        String conflictSQLSuffix = "from TXN_COMPONENTS where tc_txnid=" + txnid + " and tc_operation_type IN(" +
-          quoteChar(OpertaionType.UPDATE.sqlConst) + "," + quoteChar(OpertaionType.DELETE.sqlConst) + ")";
-        rs = stmt.executeQuery(sqlGenerator.addLimitClause(1, "tc_operation_type " + conflictSQLSuffix));
-        if (rs.next()) {
+
+        String conflictSQLSuffix = null;
+        if (rqst.isSetReplPolicy()) {
+          rs = null;
+        } else {
+          conflictSQLSuffix = "from TXN_COMPONENTS where tc_txnid=" + txnid + " and tc_operation_type IN(" +
+                  quoteChar(OpertaionType.UPDATE.sqlConst) + "," + quoteChar(OpertaionType.DELETE.sqlConst) + ")";
+          rs = stmt.executeQuery(sqlGenerator.addLimitClause(1,
+                  "tc_operation_type " + conflictSQLSuffix));
+        }
+        if (rs != null && rs.next()) {
           isUpdateDelete = true;
           close(rs);
           //if here it means currently committing txn performed update/delete and we should check WW conflict
@@ -984,23 +1024,52 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
            * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's
            * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn.
            * If RO < W, then there is no reads-from relationship.
+           * In replication flow we don't expect any write write conflict as it should have been handled at source.
            */
         }
-        // Move the record from txn_components into completed_txn_components so that the compactor
-        // knows where to look to compact.
-        String s = "insert into COMPLETED_TXN_COMPONENTS (ctc_txnid, ctc_database, " +
-            "ctc_table, ctc_partition, ctc_writeid) select tc_txnid, tc_database, tc_table, " +
-            "tc_partition, tc_writeid from TXN_COMPONENTS where tc_txnid = " + txnid;
-        LOG.debug("Going to execute insert <" + s + ">");
-        int modCount = 0;
-        if ((modCount = stmt.executeUpdate(s)) < 1) {
-          //this can be reasonable for an empty txn START/COMMIT or read-only txn
-          //also an IUD with DP that didn't match any rows.
-          LOG.info("Expected to move at least one record from txn_components to " +
-            "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
+
+        String s;
+        if (!rqst.isSetReplPolicy()) {
+          // Move the record from txn_components into completed_txn_components so that the compactor
+          // knows where to look to compact.
+          s = "insert into COMPLETED_TXN_COMPONENTS (ctc_txnid, ctc_database, " +
+                  "ctc_table, ctc_partition, ctc_writeid) select tc_txnid, tc_database, tc_table, " +
+                  "tc_partition, tc_writeid from TXN_COMPONENTS where tc_txnid = " + txnid;
+          LOG.debug("Going to execute insert <" + s + ">");
+
+          if ((stmt.executeUpdate(s)) < 1) {
+            //this can be reasonable for an empty txn START/COMMIT or read-only txn
+            //also an IUD with DP that didn't match any rows.
+            LOG.info("Expected to move at least one record from txn_components to " +
+                    "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
+          }
+        } else {
+          if (rqst.isSetWriteEventInfos()) {
+            List<String> rows = new ArrayList<>();
+            for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
+              rows.add(txnid + "," + quoteString(writeEventInfo.getDatabase()) + "," +
+                      quoteString(writeEventInfo.getTable()) + "," +
+                      quoteString(writeEventInfo.getPartition()) + "," +
+                      writeEventInfo.getWriteId());
+            }
+            List<String> queries = sqlGenerator.createInsertValuesStmt("COMPLETED_TXN_COMPONENTS " +
+                    "(ctc_txnid," + " ctc_database, ctc_table, ctc_partition, ctc_writeid)", rows);
+            for (String q : queries) {
+              LOG.debug("Going to execute insert  <" + q + "> ");
+              stmt.execute(q);
+            }
+          }
+
+          s = "delete from REPL_TXN_MAP where RTM_SRC_TXN_ID = " + sourceTxnId +
+                  " and RTM_REPL_POLICY = " + quoteString(rqst.getReplPolicy());
+          LOG.info("Repl going to execute  <" + s + ">");
+          stmt.executeUpdate(s);
         }
+
         // Obtain information that we need to update registry
-        s = "select ctc_database, ctc_table, ctc_writeid, ctc_timestamp from COMPLETED_TXN_COMPONENTS where ctc_txnid = " + txnid;
+        s = "select ctc_database, ctc_table, ctc_writeid, ctc_timestamp from COMPLETED_TXN_COMPONENTS" +
+                " where ctc_txnid = " + txnid;
+
         LOG.debug("Going to extract table modification information for invalidation cache <" + s + ">");
         rs = stmt.executeQuery(s);
         while (rs.next()) {
@@ -1008,27 +1077,21 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           txnComponents.add(new TransactionRegistryInfo(rs.getString(1), rs.getString(2),
               rs.getLong(3), rs.getTimestamp(4, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime()));
         }
+
+        // cleanup all txn related metadata
         s = "delete from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from HIVE_LOCKS where hl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from TXNS where txn_id = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from MIN_HISTORY_LEVEL where mhl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         LOG.info("Removed committed transaction: (" + txnid + ") from MIN_HISTORY_LEVEL");
-
-        if (rqst.isSetReplPolicy()) {
-          s = "delete from REPL_TXN_MAP where RTM_SRC_TXN_ID = " + sourceTxnId +
-                  " and RTM_REPL_POLICY = " + quoteString(rqst.getReplPolicy());
-          LOG.info("Repl going to execute  <" + s + ">");
-          stmt.executeUpdate(s);
-        }
-
         if (transactionalListeners != null) {
           MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
                   EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, null), dbConn, sqlGenerator);
@@ -1548,6 +1611,43 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   }
   @Override
+  @RetrySemantics.Idempotent
+  public void addWriteNotificationLog(AcidWriteEvent acidWriteEvent)
+          throws MetaException {
+    Connection dbConn = null;
+    try {
+      try {
+        //Idempotent case is handled by notify Event
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                EventMessage.EventType.ACID_WRITE, acidWriteEvent, dbConn, sqlGenerator);
+        LOG.debug("Going to commit");
+        dbConn.commit();
+        return;
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        if (isDuplicateKeyError(e)) {
+          // in case of key duplicate error, retry as it might be because of race condition
+          if (waitForRetry("addWriteNotificationLog(" + acidWriteEvent + ")", e.getMessage())) {
+            throw new RetryException();
+          }
+          retryNum = 0;
+          throw new MetaException(e.getMessage());
+        }
+        checkRetryable(dbConn, e, "addWriteNotificationLog(" + acidWriteEvent + ")");
+        throw new MetaException("Unable to add write notification event " + StringUtils.stringifyException(e));
+      } finally{
+        closeDbConn(dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      addWriteNotificationLog(acidWriteEvent);
+    }
+  }
+
+  @Override
   @RetrySemantics.SafeToRetry
   public void performWriteSetGC() {
     Connection dbConn = null;
@@ -3046,6 +3146,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     closeStmt(stmt);
     closeDbConn(dbConn);
   }
+
+  private boolean waitForRetry(String caller, String errMsg) {
+    if (retryNum++ < retryLimit) {
+      LOG.warn("Retryable error detected in " + caller + ".  Will wait " + retryInterval +
+              "ms and retry up to " + (retryLimit - retryNum + 1) + " times.  Error: " + errMsg);
+      try {
+        Thread.sleep(retryInterval);
+      } catch (InterruptedException ex) {
+        //
+      }
+      return true;
+    } else {
+      LOG.error("Fatal error in " + caller + ". Retry limit (" + retryLimit + ") reached. Last error: " + errMsg);
+    }
+    return false;
+  }
   /**
    * Determine if an exception was such that it makes sense to retry.  Unfortunately there is no standard way to do
    * this, so we have to inspect the error messages and catch the telltale signs for each
@@ -3089,18 +3205,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
       } else if (isRetryable(conf, e)) {
         //in MSSQL this means Communication Link Failure
-        if (retryNum++ < retryLimit) {
-          LOG.warn("Retryable error detected in " + caller + ".  Will wait " + retryInterval +
-            "ms and retry up to " + (retryLimit - retryNum + 1) + " times.  Error: " + getMessage(e));
-          try {
-            Thread.sleep(retryInterval);
-          } catch (InterruptedException ex) {
-            //
-          }
-          sendRetrySignal = true;
-        } else {
-          LOG.error("Fatal error in " + caller + ". Retry limit (" + retryLimit + ") reached. Last error: " + getMessage(e));
-        }
+        sendRetrySignal = waitForRetry(caller, e.getMessage());
       }
       else {
         //make sure we know we saw an error that we don't recognize

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index ef447e1..d972d10 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 
 import java.sql.SQLException;
 import java.util.Iterator;
@@ -86,6 +87,9 @@ public interface TxnStore extends Configurable {
   @RetrySemantics.Idempotent
   OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException;
 
+  @RetrySemantics.Idempotent
+  long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException;
+
   /**
    * Abort (rollback) a transaction.
    * @param rqst info on transaction to abort
@@ -476,4 +480,11 @@ public interface TxnStore extends Configurable {
    */
   @RetrySemantics.Idempotent
   void setHadoopJobId(String hadoopJobId, long id);
+
+  /**
+   * Add the ACID write event information to writeNotificationLog table.
+   * @param acidWriteEvent
+   */
+  @RetrySemantics.Idempotent
+  void addWriteNotificationLog(AcidWriteEvent acidWriteEvent) throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
index 963e12f..154db4b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
@@ -511,7 +511,6 @@ public class FileUtils {
     return new Path(scheme, authority, pathUri.getPath());
   }
 
-
   /**
    * Returns a BEST GUESS as to whether or not other is a subdirectory of parent. It does not
    * take into account any intricacies of the underlying file system, which is assumed to be
@@ -524,4 +523,15 @@ public class FileUtils {
   public static boolean isSubdirectory(String parent, String other) {
     return other.startsWith(parent.endsWith(Path.SEPARATOR) ? parent : parent + Path.SEPARATOR);
   }
+
+  public static Path getTransformedPath(String name, String subDir, String root) {
+    if (root != null) {
+      Path newPath = new Path(root);
+      if (subDir != null) {
+        newPath = new Path(newPath, subDir);
+      }
+      return new Path(newPath, name);
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 1be3e98..5fb548c 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -1182,6 +1182,41 @@
       </field>
     </class>
 
+    <class name="MTxnWriteNotificationLog" table="TXN_WRITE_NOTIFICATION_LOG" identity-type="datastore" detachable="true">
+      <datastore-identity strategy="increment"/>
+      <datastore-identity key-cache-size="1"/>
+      <datastore-identity>
+        <column name="WNL_ID"/>
+      </datastore-identity>
+      <field name="txnId">
+        <column name="WNL_TXNID" jdbc-type="BIGINT" allows-null="false"/>
+      </field>
+      <field name="writeId">
+        <column name="WNL_WRITEID" jdbc-type="BIGINT" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="WNL_DATABASE" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="table">
+        <column name="WNL_TABLE" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="partition">
+        <column name="WNL_PARTITION" length="1024" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="tableObject">
+        <column name="WNL_TABLE_OBJ" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="partObject">
+        <column name="WNL_PARTITION_OBJ" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="files">
+        <column name="WNL_FILES" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="eventTime">
+        <column name="WNL_EVENT_TIME" jdbc-type="INTEGER" allows-null="false"/>
+      </field>
+    </class>
+
     <class name="MWMResourcePlan" identity-type="datastore" table="WM_RESOURCEPLAN" detachable="true">
       <datastore-identity>
         <column name="RP_ID"/>

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
index 352b43e..a696d06 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
@@ -689,6 +689,21 @@ CREATE TABLE "APP"."RUNTIME_STATS" (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
index bb69105..7cab4fb 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
@@ -689,6 +689,21 @@ CREATE TABLE "APP"."RUNTIME_STATS" (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 7b7a8a2..10f1373 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -244,7 +244,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-
 CREATE TABLE "APP"."RUNTIME_STATS" (
   "RS_ID" bigint primary key,
   "CREATE_TIME" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
index 6621ef7..7058ab0 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
@@ -29,5 +29,21 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE integer;
 
 CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_NAME");
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
index bc11b40..d7722dc 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
@@ -1248,6 +1248,23 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
index 922e8fe..a81fc40 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
@@ -1249,6 +1249,23 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
index abb80d6..41f23f7 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD TXN_TYPE int NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME);
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index c54df55..c65af1e 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -1155,7 +1155,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-
 CREATE TABLE RUNTIME_STATS (
   RS_ID bigint primary key,
   CREATE_TIME bigint NOT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
index af955dc..29d4a43 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
@@ -1173,6 +1173,22 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
index 6c40e6e..968f4a4 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
@@ -1173,6 +1173,22 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 9b87563..786e38a 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -319,8 +319,8 @@ UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
 
 ALTER TABLE TXN_COMPONENTS MODIFY COLUMN TC_TABLE varchar(128) NULL;
 
+ALTER TABLE `TBLS` ADD COLUMN `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';
-
-ALTER TABLE `TBLS` ADD COLUMN `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
index 305fa1d..e103bef 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE int DEFAULT NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME) USING BTREE;
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index 63cc1f7..3e2e282 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -1134,7 +1134,6 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
index bc13703..9adea31 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
@@ -1143,6 +1143,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
index e12150a..faca669 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
@@ -1143,6 +1143,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index ce3437f..71f5034 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -335,8 +335,8 @@ INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
 UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
 UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
 
+ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;
-
-ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
index ccdea54..cf8699b 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD TXN_TYPE number(10) NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME);
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 97697f8..b89c87f 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -1812,7 +1812,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 );
 
-
 CREATE TABLE RUNTIME_STATS (
  RS_ID bigint primary key,
  CREATE_TIME bigint NOT NULL,
@@ -1822,7 +1821,6 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
index 36bab70..7a8a419 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
@@ -1834,6 +1834,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
index b73e1d1..2e7ac5a 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
@@ -1836,6 +1836,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
index 2c0eb31..445c3a2 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
@@ -32,6 +32,22 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE integer DEFAULT NULL;
 
 CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME");
 
+-- HIVE-19267
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.1.0', "VERSION_COMMENT"='Hive release version 3.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 6e503eb..1ca6454 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -867,6 +867,18 @@ struct AbortTxnsRequest {
 struct CommitTxnRequest {
     1: required i64 txnid,
     2: optional string replPolicy,
+    // Information related to write operations done in this transaction.
+    3: optional list<WriteEventInfo> writeEventInfos,
+}
+
+struct WriteEventInfo {
+    1: required i64    writeId,
+    2: required string database,
+    3: required string table,
+    4: required string files,
+    5: optional string partition,
+    6: optional string tableObj, // repl txn task does not need table object for commit
+    7: optional string partitionObj,
 }
 
 struct ReplTblWriteIdStateRequest {
@@ -1102,6 +1114,8 @@ struct InsertEventRequestData {
     2: required list<string> filesAdded,
     // Checksum of files (hex string of checksum byte payload)
     3: optional list<string> filesAddedChecksum,
+    // Used by acid operation to create the sub directory
+    4: optional list<string> subDirectoryList,
 }
 
 union FireEventRequestData {
@@ -1122,7 +1136,20 @@ struct FireEventRequest {
 struct FireEventResponse {
     // NOP for now, this is just a place holder for future responses
 }
-    
+
+struct WriteNotificationLogRequest {
+    1: required i64 txnId,
+    2: required i64 writeId,
+    3: required string db,
+    4: required string table,
+    5: required InsertEventRequestData fileInfo,
+    6: optional list<string> partitionVals,
+}
+
+struct WriteNotificationLogResponse {
+    // NOP for now, this is just a place holder for future responses
+}
+
 struct MetadataPpdResult {
   1: optional binary metadata,
   2: optional binary includeBitset
@@ -2104,6 +2131,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   NotificationEventsCountResponse get_notification_events_count(1:NotificationEventsCountRequest rqst)
   FireEventResponse fire_listener_event(1:FireEventRequest rqst)
   void flushCache()
+  WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)
 
   // Repl Change Management api
   CmRecycleResponse cm_recycle(1:CmRecycleRequest request) throws(1:MetaException o1)

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index c482d28..2454479 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -1195,6 +1196,16 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    objectStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return objectStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
+  @Override
   public List<TableName> getTableNamesWithStats() throws MetaException,
       NoSuchObjectException {
     return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index d253005..9b79446 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
@@ -1199,4 +1200,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       NoSuchObjectException {
     return null;
   }
+
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return null;
+  }
 }


[09/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 672ebf9..ec129ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -372,6 +372,8 @@ import org.slf4j.LoggerFactory;
 
     public void flushCache() throws org.apache.thrift.TException;
 
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException;
+
     public CmRecycleResponse cm_recycle(CmRecycleRequest request) throws MetaException, org.apache.thrift.TException;
 
     public GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req) throws org.apache.thrift.TException;
@@ -790,6 +792,8 @@ import org.slf4j.LoggerFactory;
 
     public void flushCache(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void add_write_notification_log(WriteNotificationLogRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void cm_recycle(CmRecycleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_file_metadata_by_expr(GetFileMetadataByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -5644,6 +5648,29 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException
+    {
+      send_add_write_notification_log(rqst);
+      return recv_add_write_notification_log();
+    }
+
+    public void send_add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException
+    {
+      add_write_notification_log_args args = new add_write_notification_log_args();
+      args.setRqst(rqst);
+      sendBase("add_write_notification_log", args);
+    }
+
+    public WriteNotificationLogResponse recv_add_write_notification_log() throws org.apache.thrift.TException
+    {
+      add_write_notification_log_result result = new add_write_notification_log_result();
+      receiveBase(result, "add_write_notification_log");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "add_write_notification_log failed: unknown result");
+    }
+
     public CmRecycleResponse cm_recycle(CmRecycleRequest request) throws MetaException, org.apache.thrift.TException
     {
       send_cm_recycle(request);
@@ -12619,6 +12646,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void add_write_notification_log(WriteNotificationLogRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_write_notification_log_call method_call = new add_write_notification_log_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private WriteNotificationLogRequest rqst;
+      public add_write_notification_log_call(WriteNotificationLogRequest 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("add_write_notification_log", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_write_notification_log_args args = new add_write_notification_log_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public WriteNotificationLogResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_add_write_notification_log();
+      }
+    }
+
     public void cm_recycle(CmRecycleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cm_recycle_call method_call = new cm_recycle_call(request, resultHandler, this, ___protocolFactory, ___transport);
@@ -14150,6 +14209,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_notification_events_count", new get_notification_events_count());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("add_write_notification_log", new add_write_notification_log());
       processMap.put("cm_recycle", new cm_recycle());
       processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
       processMap.put("get_file_metadata", new get_file_metadata());
@@ -18426,6 +18486,26 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_write_notification_log_args> {
+      public add_write_notification_log() {
+        super("add_write_notification_log");
+      }
+
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_write_notification_log_result getResult(I iface, add_write_notification_log_args args) throws org.apache.thrift.TException {
+        add_write_notification_log_result result = new add_write_notification_log_result();
+        result.success = iface.add_write_notification_log(args.rqst);
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cm_recycle_args> {
       public cm_recycle() {
         super("cm_recycle");
@@ -19683,6 +19763,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_notification_events_count", new get_notification_events_count());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("add_write_notification_log", new add_write_notification_log());
       processMap.put("cm_recycle", new cm_recycle());
       processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
       processMap.put("get_file_metadata", new get_file_metadata());
@@ -29810,6 +29891,57 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_args, WriteNotificationLogResponse> {
+      public add_write_notification_log() {
+        super("add_write_notification_log");
+      }
+
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_args();
+      }
+
+      public AsyncMethodCallback<WriteNotificationLogResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<WriteNotificationLogResponse>() { 
+          public void onComplete(WriteNotificationLogResponse o) {
+            add_write_notification_log_result result = new add_write_notification_log_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            add_write_notification_log_result result = new add_write_notification_log_result();
+            {
+              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, add_write_notification_log_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> resultHandler) throws TException {
+        iface.add_write_notification_log(args.rqst,resultHandler);
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cm_recycle_args, CmRecycleResponse> {
       public cm_recycle() {
         super("cm_recycle");
@@ -42252,13 +42384,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list928.size);
-                  String _elem929;
-                  for (int _i930 = 0; _i930 < _list928.size; ++_i930)
+                  org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list952.size);
+                  String _elem953;
+                  for (int _i954 = 0; _i954 < _list952.size; ++_i954)
                   {
-                    _elem929 = iprot.readString();
-                    struct.success.add(_elem929);
+                    _elem953 = iprot.readString();
+                    struct.success.add(_elem953);
                   }
                   iprot.readListEnd();
                 }
@@ -42293,9 +42425,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter931 : struct.success)
+            for (String _iter955 : struct.success)
             {
-              oprot.writeString(_iter931);
+              oprot.writeString(_iter955);
             }
             oprot.writeListEnd();
           }
@@ -42334,9 +42466,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter932 : struct.success)
+            for (String _iter956 : struct.success)
             {
-              oprot.writeString(_iter932);
+              oprot.writeString(_iter956);
             }
           }
         }
@@ -42351,13 +42483,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list933.size);
-            String _elem934;
-            for (int _i935 = 0; _i935 < _list933.size; ++_i935)
+            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list957.size);
+            String _elem958;
+            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
             {
-              _elem934 = iprot.readString();
-              struct.success.add(_elem934);
+              _elem958 = iprot.readString();
+              struct.success.add(_elem958);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43011,13 +43143,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list936.size);
-                  String _elem937;
-                  for (int _i938 = 0; _i938 < _list936.size; ++_i938)
+                  org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list960.size);
+                  String _elem961;
+                  for (int _i962 = 0; _i962 < _list960.size; ++_i962)
                   {
-                    _elem937 = iprot.readString();
-                    struct.success.add(_elem937);
+                    _elem961 = iprot.readString();
+                    struct.success.add(_elem961);
                   }
                   iprot.readListEnd();
                 }
@@ -43052,9 +43184,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter939 : struct.success)
+            for (String _iter963 : struct.success)
             {
-              oprot.writeString(_iter939);
+              oprot.writeString(_iter963);
             }
             oprot.writeListEnd();
           }
@@ -43093,9 +43225,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter940 : struct.success)
+            for (String _iter964 : struct.success)
             {
-              oprot.writeString(_iter940);
+              oprot.writeString(_iter964);
             }
           }
         }
@@ -43110,13 +43242,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list941.size);
-            String _elem942;
-            for (int _i943 = 0; _i943 < _list941.size; ++_i943)
+            org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list965.size);
+            String _elem966;
+            for (int _i967 = 0; _i967 < _list965.size; ++_i967)
             {
-              _elem942 = iprot.readString();
-              struct.success.add(_elem942);
+              _elem966 = iprot.readString();
+              struct.success.add(_elem966);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47723,16 +47855,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map944 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map944.size);
-                  String _key945;
-                  Type _val946;
-                  for (int _i947 = 0; _i947 < _map944.size; ++_i947)
+                  org.apache.thrift.protocol.TMap _map968 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map968.size);
+                  String _key969;
+                  Type _val970;
+                  for (int _i971 = 0; _i971 < _map968.size; ++_i971)
                   {
-                    _key945 = iprot.readString();
-                    _val946 = new Type();
-                    _val946.read(iprot);
-                    struct.success.put(_key945, _val946);
+                    _key969 = iprot.readString();
+                    _val970 = new Type();
+                    _val970.read(iprot);
+                    struct.success.put(_key969, _val970);
                   }
                   iprot.readMapEnd();
                 }
@@ -47767,10 +47899,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter948 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter972 : struct.success.entrySet())
             {
-              oprot.writeString(_iter948.getKey());
-              _iter948.getValue().write(oprot);
+              oprot.writeString(_iter972.getKey());
+              _iter972.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -47809,10 +47941,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter949 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter973 : struct.success.entrySet())
             {
-              oprot.writeString(_iter949.getKey());
-              _iter949.getValue().write(oprot);
+              oprot.writeString(_iter973.getKey());
+              _iter973.getValue().write(oprot);
             }
           }
         }
@@ -47827,16 +47959,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map950 = 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*_map950.size);
-            String _key951;
-            Type _val952;
-            for (int _i953 = 0; _i953 < _map950.size; ++_i953)
+            org.apache.thrift.protocol.TMap _map974 = 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*_map974.size);
+            String _key975;
+            Type _val976;
+            for (int _i977 = 0; _i977 < _map974.size; ++_i977)
             {
-              _key951 = iprot.readString();
-              _val952 = new Type();
-              _val952.read(iprot);
-              struct.success.put(_key951, _val952);
+              _key975 = iprot.readString();
+              _val976 = new Type();
+              _val976.read(iprot);
+              struct.success.put(_key975, _val976);
             }
           }
           struct.setSuccessIsSet(true);
@@ -48871,14 +49003,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list954 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list954.size);
-                  FieldSchema _elem955;
-                  for (int _i956 = 0; _i956 < _list954.size; ++_i956)
+                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list978.size);
+                  FieldSchema _elem979;
+                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
                   {
-                    _elem955 = new FieldSchema();
-                    _elem955.read(iprot);
-                    struct.success.add(_elem955);
+                    _elem979 = new FieldSchema();
+                    _elem979.read(iprot);
+                    struct.success.add(_elem979);
                   }
                   iprot.readListEnd();
                 }
@@ -48931,9 +49063,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter957 : struct.success)
+            for (FieldSchema _iter981 : struct.success)
             {
-              _iter957.write(oprot);
+              _iter981.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -48988,9 +49120,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter958 : struct.success)
+            for (FieldSchema _iter982 : struct.success)
             {
-              _iter958.write(oprot);
+              _iter982.write(oprot);
             }
           }
         }
@@ -49011,14 +49143,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list959 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list959.size);
-            FieldSchema _elem960;
-            for (int _i961 = 0; _i961 < _list959.size; ++_i961)
+            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list983.size);
+            FieldSchema _elem984;
+            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
             {
-              _elem960 = new FieldSchema();
-              _elem960.read(iprot);
-              struct.success.add(_elem960);
+              _elem984 = new FieldSchema();
+              _elem984.read(iprot);
+              struct.success.add(_elem984);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50172,14 +50304,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list962 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list962.size);
-                  FieldSchema _elem963;
-                  for (int _i964 = 0; _i964 < _list962.size; ++_i964)
+                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list986.size);
+                  FieldSchema _elem987;
+                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
                   {
-                    _elem963 = new FieldSchema();
-                    _elem963.read(iprot);
-                    struct.success.add(_elem963);
+                    _elem987 = new FieldSchema();
+                    _elem987.read(iprot);
+                    struct.success.add(_elem987);
                   }
                   iprot.readListEnd();
                 }
@@ -50232,9 +50364,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter965 : struct.success)
+            for (FieldSchema _iter989 : struct.success)
             {
-              _iter965.write(oprot);
+              _iter989.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -50289,9 +50421,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter966 : struct.success)
+            for (FieldSchema _iter990 : struct.success)
             {
-              _iter966.write(oprot);
+              _iter990.write(oprot);
             }
           }
         }
@@ -50312,14 +50444,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list967 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list967.size);
-            FieldSchema _elem968;
-            for (int _i969 = 0; _i969 < _list967.size; ++_i969)
+            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list991.size);
+            FieldSchema _elem992;
+            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
             {
-              _elem968 = new FieldSchema();
-              _elem968.read(iprot);
-              struct.success.add(_elem968);
+              _elem992 = new FieldSchema();
+              _elem992.read(iprot);
+              struct.success.add(_elem992);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51364,14 +51496,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list970.size);
-                  FieldSchema _elem971;
-                  for (int _i972 = 0; _i972 < _list970.size; ++_i972)
+                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list994.size);
+                  FieldSchema _elem995;
+                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
                   {
-                    _elem971 = new FieldSchema();
-                    _elem971.read(iprot);
-                    struct.success.add(_elem971);
+                    _elem995 = new FieldSchema();
+                    _elem995.read(iprot);
+                    struct.success.add(_elem995);
                   }
                   iprot.readListEnd();
                 }
@@ -51424,9 +51556,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter973 : struct.success)
+            for (FieldSchema _iter997 : struct.success)
             {
-              _iter973.write(oprot);
+              _iter997.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51481,9 +51613,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter974 : struct.success)
+            for (FieldSchema _iter998 : struct.success)
             {
-              _iter974.write(oprot);
+              _iter998.write(oprot);
             }
           }
         }
@@ -51504,14 +51636,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list975 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list975.size);
-            FieldSchema _elem976;
-            for (int _i977 = 0; _i977 < _list975.size; ++_i977)
+            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list999.size);
+            FieldSchema _elem1000;
+            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
             {
-              _elem976 = new FieldSchema();
-              _elem976.read(iprot);
-              struct.success.add(_elem976);
+              _elem1000 = new FieldSchema();
+              _elem1000.read(iprot);
+              struct.success.add(_elem1000);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52665,14 +52797,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list978.size);
-                  FieldSchema _elem979;
-                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
+                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
+                  FieldSchema _elem1003;
+                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
                   {
-                    _elem979 = new FieldSchema();
-                    _elem979.read(iprot);
-                    struct.success.add(_elem979);
+                    _elem1003 = new FieldSchema();
+                    _elem1003.read(iprot);
+                    struct.success.add(_elem1003);
                   }
                   iprot.readListEnd();
                 }
@@ -52725,9 +52857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter981 : struct.success)
+            for (FieldSchema _iter1005 : struct.success)
             {
-              _iter981.write(oprot);
+              _iter1005.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52782,9 +52914,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter982 : struct.success)
+            for (FieldSchema _iter1006 : struct.success)
             {
-              _iter982.write(oprot);
+              _iter1006.write(oprot);
             }
           }
         }
@@ -52805,14 +52937,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list983.size);
-            FieldSchema _elem984;
-            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
+            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1007.size);
+            FieldSchema _elem1008;
+            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
             {
-              _elem984 = new FieldSchema();
-              _elem984.read(iprot);
-              struct.success.add(_elem984);
+              _elem1008 = new FieldSchema();
+              _elem1008.read(iprot);
+              struct.success.add(_elem1008);
             }
           }
           struct.setSuccessIsSet(true);
@@ -55941,14 +56073,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list986.size);
-                  SQLPrimaryKey _elem987;
-                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
+                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1010.size);
+                  SQLPrimaryKey _elem1011;
+                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
                   {
-                    _elem987 = new SQLPrimaryKey();
-                    _elem987.read(iprot);
-                    struct.primaryKeys.add(_elem987);
+                    _elem1011 = new SQLPrimaryKey();
+                    _elem1011.read(iprot);
+                    struct.primaryKeys.add(_elem1011);
                   }
                   iprot.readListEnd();
                 }
@@ -55960,14 +56092,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list989 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list989.size);
-                  SQLForeignKey _elem990;
-                  for (int _i991 = 0; _i991 < _list989.size; ++_i991)
+                  org.apache.thrift.protocol.TList _list1013 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1013.size);
+                  SQLForeignKey _elem1014;
+                  for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
                   {
-                    _elem990 = new SQLForeignKey();
-                    _elem990.read(iprot);
-                    struct.foreignKeys.add(_elem990);
+                    _elem1014 = new SQLForeignKey();
+                    _elem1014.read(iprot);
+                    struct.foreignKeys.add(_elem1014);
                   }
                   iprot.readListEnd();
                 }
@@ -55979,14 +56111,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list992 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list992.size);
-                  SQLUniqueConstraint _elem993;
-                  for (int _i994 = 0; _i994 < _list992.size; ++_i994)
+                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1016.size);
+                  SQLUniqueConstraint _elem1017;
+                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
                   {
-                    _elem993 = new SQLUniqueConstraint();
-                    _elem993.read(iprot);
-                    struct.uniqueConstraints.add(_elem993);
+                    _elem1017 = new SQLUniqueConstraint();
+                    _elem1017.read(iprot);
+                    struct.uniqueConstraints.add(_elem1017);
                   }
                   iprot.readListEnd();
                 }
@@ -55998,14 +56130,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list995 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list995.size);
-                  SQLNotNullConstraint _elem996;
-                  for (int _i997 = 0; _i997 < _list995.size; ++_i997)
+                  org.apache.thrift.protocol.TList _list1019 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1019.size);
+                  SQLNotNullConstraint _elem1020;
+                  for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
                   {
-                    _elem996 = new SQLNotNullConstraint();
-                    _elem996.read(iprot);
-                    struct.notNullConstraints.add(_elem996);
+                    _elem1020 = new SQLNotNullConstraint();
+                    _elem1020.read(iprot);
+                    struct.notNullConstraints.add(_elem1020);
                   }
                   iprot.readListEnd();
                 }
@@ -56017,14 +56149,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list998 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list998.size);
-                  SQLDefaultConstraint _elem999;
-                  for (int _i1000 = 0; _i1000 < _list998.size; ++_i1000)
+                  org.apache.thrift.protocol.TList _list1022 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1022.size);
+                  SQLDefaultConstraint _elem1023;
+                  for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
                   {
-                    _elem999 = new SQLDefaultConstraint();
-                    _elem999.read(iprot);
-                    struct.defaultConstraints.add(_elem999);
+                    _elem1023 = new SQLDefaultConstraint();
+                    _elem1023.read(iprot);
+                    struct.defaultConstraints.add(_elem1023);
                   }
                   iprot.readListEnd();
                 }
@@ -56036,14 +56168,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1001 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1001.size);
-                  SQLCheckConstraint _elem1002;
-                  for (int _i1003 = 0; _i1003 < _list1001.size; ++_i1003)
+                  org.apache.thrift.protocol.TList _list1025 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1025.size);
+                  SQLCheckConstraint _elem1026;
+                  for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
                   {
-                    _elem1002 = new SQLCheckConstraint();
-                    _elem1002.read(iprot);
-                    struct.checkConstraints.add(_elem1002);
+                    _elem1026 = new SQLCheckConstraint();
+                    _elem1026.read(iprot);
+                    struct.checkConstraints.add(_elem1026);
                   }
                   iprot.readListEnd();
                 }
@@ -56074,9 +56206,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1004 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1028 : struct.primaryKeys)
             {
-              _iter1004.write(oprot);
+              _iter1028.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56086,9 +56218,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1005 : struct.foreignKeys)
+            for (SQLForeignKey _iter1029 : struct.foreignKeys)
             {
-              _iter1005.write(oprot);
+              _iter1029.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56098,9 +56230,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1006 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1030 : struct.uniqueConstraints)
             {
-              _iter1006.write(oprot);
+              _iter1030.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56110,9 +56242,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1007 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1031 : struct.notNullConstraints)
             {
-              _iter1007.write(oprot);
+              _iter1031.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56122,9 +56254,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1008 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1032 : struct.defaultConstraints)
             {
-              _iter1008.write(oprot);
+              _iter1032.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56134,9 +56266,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1009 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1033 : struct.checkConstraints)
             {
-              _iter1009.write(oprot);
+              _iter1033.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56188,54 +56320,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1010 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1034 : struct.primaryKeys)
             {
-              _iter1010.write(oprot);
+              _iter1034.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1011 : struct.foreignKeys)
+            for (SQLForeignKey _iter1035 : struct.foreignKeys)
             {
-              _iter1011.write(oprot);
+              _iter1035.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1012 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1036 : struct.uniqueConstraints)
             {
-              _iter1012.write(oprot);
+              _iter1036.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1013 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1037 : struct.notNullConstraints)
             {
-              _iter1013.write(oprot);
+              _iter1037.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1014 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1038 : struct.defaultConstraints)
             {
-              _iter1014.write(oprot);
+              _iter1038.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1015 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1039 : struct.checkConstraints)
             {
-              _iter1015.write(oprot);
+              _iter1039.write(oprot);
             }
           }
         }
@@ -56252,84 +56384,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1016 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1016.size);
-            SQLPrimaryKey _elem1017;
-            for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
+            org.apache.thrift.protocol.TList _list1040 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1040.size);
+            SQLPrimaryKey _elem1041;
+            for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
             {
-              _elem1017 = new SQLPrimaryKey();
-              _elem1017.read(iprot);
-              struct.primaryKeys.add(_elem1017);
+              _elem1041 = new SQLPrimaryKey();
+              _elem1041.read(iprot);
+              struct.primaryKeys.add(_elem1041);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1019 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1019.size);
-            SQLForeignKey _elem1020;
-            for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
+            org.apache.thrift.protocol.TList _list1043 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1043.size);
+            SQLForeignKey _elem1044;
+            for (int _i1045 = 0; _i1045 < _list1043.size; ++_i1045)
             {
-              _elem1020 = new SQLForeignKey();
-              _elem1020.read(iprot);
-              struct.foreignKeys.add(_elem1020);
+              _elem1044 = new SQLForeignKey();
+              _elem1044.read(iprot);
+              struct.foreignKeys.add(_elem1044);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1022 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1022.size);
-            SQLUniqueConstraint _elem1023;
-            for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
+            org.apache.thrift.protocol.TList _list1046 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1046.size);
+            SQLUniqueConstraint _elem1047;
+            for (int _i1048 = 0; _i1048 < _list1046.size; ++_i1048)
             {
-              _elem1023 = new SQLUniqueConstraint();
-              _elem1023.read(iprot);
-              struct.uniqueConstraints.add(_elem1023);
+              _elem1047 = new SQLUniqueConstraint();
+              _elem1047.read(iprot);
+              struct.uniqueConstraints.add(_elem1047);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1025 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1025.size);
-            SQLNotNullConstraint _elem1026;
-            for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
+            org.apache.thrift.protocol.TList _list1049 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1049.size);
+            SQLNotNullConstraint _elem1050;
+            for (int _i1051 = 0; _i1051 < _list1049.size; ++_i1051)
             {
-              _elem1026 = new SQLNotNullConstraint();
-              _elem1026.read(iprot);
-              struct.notNullConstraints.add(_elem1026);
+              _elem1050 = new SQLNotNullConstraint();
+              _elem1050.read(iprot);
+              struct.notNullConstraints.add(_elem1050);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1028 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1028.size);
-            SQLDefaultConstraint _elem1029;
-            for (int _i1030 = 0; _i1030 < _list1028.size; ++_i1030)
+            org.apache.thrift.protocol.TList _list1052 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1052.size);
+            SQLDefaultConstraint _elem1053;
+            for (int _i1054 = 0; _i1054 < _list1052.size; ++_i1054)
             {
-              _elem1029 = new SQLDefaultConstraint();
-              _elem1029.read(iprot);
-              struct.defaultConstraints.add(_elem1029);
+              _elem1053 = new SQLDefaultConstraint();
+              _elem1053.read(iprot);
+              struct.defaultConstraints.add(_elem1053);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1031 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1031.size);
-            SQLCheckConstraint _elem1032;
-            for (int _i1033 = 0; _i1033 < _list1031.size; ++_i1033)
+            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1055.size);
+            SQLCheckConstraint _elem1056;
+            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
             {
-              _elem1032 = new SQLCheckConstraint();
-              _elem1032.read(iprot);
-              struct.checkConstraints.add(_elem1032);
+              _elem1056 = new SQLCheckConstraint();
+              _elem1056.read(iprot);
+              struct.checkConstraints.add(_elem1056);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -65479,13 +65611,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1034 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1034.size);
-                  String _elem1035;
-                  for (int _i1036 = 0; _i1036 < _list1034.size; ++_i1036)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1058.size);
+                  String _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1035 = iprot.readString();
-                    struct.partNames.add(_elem1035);
+                    _elem1059 = iprot.readString();
+                    struct.partNames.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -65521,9 +65653,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter1037 : struct.partNames)
+            for (String _iter1061 : struct.partNames)
             {
-              oprot.writeString(_iter1037);
+              oprot.writeString(_iter1061);
             }
             oprot.writeListEnd();
           }
@@ -65566,9 +65698,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1038 : struct.partNames)
+            for (String _iter1062 : struct.partNames)
             {
-              oprot.writeString(_iter1038);
+              oprot.writeString(_iter1062);
             }
           }
         }
@@ -65588,13 +65720,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1039.size);
-            String _elem1040;
-            for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
+            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1063.size);
+            String _elem1064;
+            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
             {
-              _elem1040 = iprot.readString();
-              struct.partNames.add(_elem1040);
+              _elem1064 = iprot.readString();
+              struct.partNames.add(_elem1064);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -66819,13 +66951,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1042.size);
-                  String _elem1043;
-                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
+                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1066.size);
+                  String _elem1067;
+                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                   {
-                    _elem1043 = iprot.readString();
-                    struct.success.add(_elem1043);
+                    _elem1067 = iprot.readString();
+                    struct.success.add(_elem1067);
                   }
                   iprot.readListEnd();
                 }
@@ -66860,9 +66992,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1045 : struct.success)
+            for (String _iter1069 : struct.success)
             {
-              oprot.writeString(_iter1045);
+              oprot.writeString(_iter1069);
             }
             oprot.writeListEnd();
           }
@@ -66901,9 +67033,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1046 : struct.success)
+            for (String _iter1070 : struct.success)
             {
-              oprot.writeString(_iter1046);
+              oprot.writeString(_iter1070);
             }
           }
         }
@@ -66918,13 +67050,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1047.size);
-            String _elem1048;
-            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
+            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1071.size);
+            String _elem1072;
+            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
             {
-              _elem1048 = iprot.readString();
-              struct.success.add(_elem1048);
+              _elem1072 = iprot.readString();
+              struct.success.add(_elem1072);
             }
           }
           struct.setSuccessIsSet(true);
@@ -67898,13 +68030,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1050.size);
-                  String _elem1051;
-                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
+                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1074.size);
+                  String _elem1075;
+                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                   {
-                    _elem1051 = iprot.readString();
-                    struct.success.add(_elem1051);
+                    _elem1075 = iprot.readString();
+                    struct.success.add(_elem1075);
                   }
                   iprot.readListEnd();
                 }
@@ -67939,9 +68071,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1053 : struct.success)
+            for (String _iter1077 : struct.success)
             {
-              oprot.writeString(_iter1053);
+              oprot.writeString(_iter1077);
             }
             oprot.writeListEnd();
           }
@@ -67980,9 +68112,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1054 : struct.success)
+            for (String _iter1078 : struct.success)
             {
-              oprot.writeString(_iter1054);
+              oprot.writeString(_iter1078);
             }
           }
         }
@@ -67997,13 +68129,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1055.size);
-            String _elem1056;
-            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
+            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1079.size);
+            String _elem1080;
+            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
             {
-              _elem1056 = iprot.readString();
-              struct.success.add(_elem1056);
+              _elem1080 = iprot.readString();
+              struct.success.add(_elem1080);
             }
           }
           struct.setSuccessIsSet(true);
@@ -68769,13 +68901,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1058.size);
-                  String _elem1059;
-                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
+                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1082.size);
+                  String _elem1083;
+                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                   {
-                    _elem1059 = iprot.readString();
-                    struct.success.add(_elem1059);
+                    _elem1083 = iprot.readString();
+                    struct.success.add(_elem1083);
                   }
                   iprot.readListEnd();
                 }
@@ -68810,9 +68942,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1061 : struct.success)
+            for (String _iter1085 : struct.success)
             {
-              oprot.writeString(_iter1061);
+              oprot.writeString(_iter1085);
             }
             oprot.writeListEnd();
           }
@@ -68851,9 +68983,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1062 : struct.success)
+            for (String _iter1086 : struct.success)
             {
-              oprot.writeString(_iter1062);
+              oprot.writeString(_iter1086);
             }
           }
         }
@@ -68868,13 +69000,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1063.size);
-            String _elem1064;
-            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
+            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1087.size);
+            String _elem1088;
+            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
             {
-              _elem1064 = iprot.readString();
-              struct.success.add(_elem1064);
+              _elem1088 = iprot.readString();
+              struct.success.add(_elem1088);
             }
           }
           struct.setSuccessIsSet(true);
@@ -69379,13 +69511,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1066.size);
-                  String _elem1067;
-                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
+                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1090.size);
+                  String _elem1091;
+                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                   {
-                    _elem1067 = iprot.readString();
-                    struct.tbl_types.add(_elem1067);
+                    _elem1091 = iprot.readString();
+                    struct.tbl_types.add(_elem1091);
                   }
                   iprot.readListEnd();
                 }
@@ -69421,9 +69553,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter1069 : struct.tbl_types)
+            for (String _iter1093 : struct.tbl_types)
             {
-              oprot.writeString(_iter1069);
+              oprot.writeString(_iter1093);
             }
             oprot.writeListEnd();
           }
@@ -69466,9 +69598,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1070 : struct.tbl_types)
+            for (String _iter1094 : struct.tbl_types)
             {
-              oprot.writeString(_iter1070);
+              oprot.writeString(_iter1094);
             }
           }
         }
@@ -69488,13 +69620,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1071.size);
-            String _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1095.size);
+            String _elem1096;
+            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
             {
-              _elem1072 = iprot.readString();
-              struct.tbl_types.add(_elem1072);
+              _elem1096 = iprot.readString();
+              struct.tbl_types.add(_elem1096);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -69900,14 +70032,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1074.size);
-                  TableMeta _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1098.size);
+                  TableMeta _elem1099;
+                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
                   {
-                    _elem1075 = new TableMeta();
-                    _elem1075.read(iprot);
-                    struct.success.add(_elem1075);
+                    _elem1099 = new TableMeta();
+                    _elem1099.read(iprot);
+                    struct.success.add(_elem1099);
                   }
                   iprot.readListEnd();
                 }
@@ -69942,9 +70074,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1077 : struct.success)
+            for (TableMeta _iter1101 : struct.success)
             {
-              _iter1077.write(oprot);
+              _iter1101.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -69983,9 +70115,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1078 : struct.success)
+            for (TableMeta _iter1102 : struct.success)
             {
-              _iter1078.write(oprot);
+              _iter1102.write(oprot);
             }
           }
         }
@@ -70000,14 +70132,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1079.size);
-            TableMeta _elem1080;
-            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
+            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1103.size);
+            TableMeta _elem1104;
+            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
             {
-              _elem1080 = new TableMeta();
-              _elem1080.read(iprot);
-              struct.success.add(_elem1080);
+              _elem1104 = new TableMeta();
+              _elem1104.read(iprot);
+              struct.success.add(_elem1104);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70773,13 +70905,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1082.size);
-                  String _elem1083;
-                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
+                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1106.size);
+                  String _elem1107;
+                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
                   {
-                    _elem1083 = iprot.readString();
-                    struct.success.add(_elem1083);
+                    _elem1107 = iprot.readString();
+                    struct.success.add(_elem1107);
                   }
                   iprot.readListEnd();
                 }
@@ -70814,9 +70946,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1085 : struct.success)
+            for (String _iter1109 : struct.success)
             {
-              oprot.writeString(_iter1085);
+              oprot.writeString(_iter1109);
             }
             oprot.writeListEnd();
           }
@@ -70855,9 +70987,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1086 : struct.success)
+            for (String _iter1110 : struct.success)
             {
-              oprot.writeString(_iter1086);
+              oprot.writeString(_iter1110);
             }
           }
         }
@@ -70872,13 +71004,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1087.size);
-            String _elem1088;
-            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
+            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1111.size);
+            String _elem1112;
+            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
             {
-              _elem1088 = iprot.readString();
-              struct.success.add(_elem1088);
+              _elem1112 = iprot.readString();
+              struct.success.add(_elem1112);
             }
           }
           struct.setSuccessIsSet(true);
@@ -72331,13 +72463,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1090.size);
-                  String _elem1091;
-                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
+                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1114.size);
+                  String _elem1115;
+                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
                   {
-                    _elem1091 = iprot.readString();
-                    struct.tbl_names.add(_elem1091);
+                    _elem1115 = iprot.readString();
+                    struct.tbl_names.add(_elem1115);
                   }
                   iprot.readListEnd();
                 }
@@ -72368,9 +72500,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1093 : struct.tbl_names)
+            for (String _iter1117 : struct.tbl_names)
             {
-              oprot.writeString(_iter1093);
+              oprot.writeString(_iter1117);
             }
             oprot.writeListEnd();
           }
@@ -72407,9 +72539,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1094 : struct.tbl_names)
+            for (String _iter1118 : struct.tbl_names)
             {
-              oprot.writeString(_iter1094);
+              oprot.writeString(_iter1118);
             }
           }
         }
@@ -72425,13 +72557,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1095.size);
-            String _elem1096;
-            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
+            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1119.size);
+            String _elem1120;
+            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
             {
-              _elem1096 = iprot.readString();
-              struct.tbl_names.add(_elem1096);
+              _elem1120 = iprot.readString();
+              struct.tbl_names.add(_elem1120);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -72756,14 +72888,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1098.size);
-                  Table _elem1099;
-                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
+                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1122.size);
+                  Table _elem1123;
+                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
                   {
-                    _elem1099 = new Table();
-                    _elem1099.read(iprot);
-                    struct.success.add(_elem1099);
+                    _elem1123 = new Table();
+                    _elem1123.read(iprot);
+                    struct.success.add(_elem1123);
                   }
                   iprot.readListEnd();
                 }
@@ -72789,9 +72921,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1101 : struct.success)
+            for (Table _iter1125 : struct.success)
             {
-              _iter1101.write(oprot);
+              _iter1125.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -72822,9 +72954,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1102 : struct.success)
+            for (Table _iter1126 : struct.success)
             {
-              _iter1102.write(oprot);
+              _iter1126.write(oprot);
             }
           }
         }
@@ -72836,14 +72968,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1103.size);
-            Table _elem1104;
-            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
+            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1127.size);
+            Table _elem1128;
+            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
             {
-              _elem1104 = new Table();
-              _elem1104.read(iprot);
-              struct.success.add(_elem1104);
+              _elem1128 = new Table();
+              _elem1128.read(iprot);
+              struct.success.add(_elem1128);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75236,13 +75368,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1106.size);
-                  String _elem1107;
-                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
+                  org.apache.thrift.protocol.TList _list1130 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1130.size);
+                  String _elem1131;
+                  for (int _i1132 = 0; _i1132 < _list1130.size; ++_i1132)
                   {
-                    _elem1107 = iprot.readString();
-                    struct.tbl_names.add(_elem1107);
+                    _elem1131 = iprot.readString();
+                    struct.tbl_names.add(_elem1131);
                   }
                   iprot.readListEnd();
                 }
@@ -75273,9 +75405,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1109 : struct.tbl_names)
+            for (String _iter1133 : struct.tbl_names)
             {
-              oprot.writeString(_iter1109);
+              oprot.writeString(_iter1133);
             }
             oprot.writeListEnd();
           }
@@ -75312,9 +75444,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1110 : struct.tbl_names)
+            for (String _iter1134 : struct.tbl_names)
             {
-              oprot.writeString(_iter1110);
+              oprot.writeString(_iter1134);
             }
           }
         }
@@ -75330,13 +75462,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1111.size);
-            String _elem1112;
-            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
+            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1135.size);
+            String _elem1136;
+            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
             {
-              _elem1112 = iprot.readString();
-              struct.tbl_names.add(_elem1112);
+              _elem1136 = iprot.readString();
+              struct.tbl_names.add(_elem1136);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -75909,16 +76041,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1114 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Materialization>(2*_map1114.size);
-                  String _key1115;
-                  Materialization _val1116;
-                  for (int _i1117 = 0; _i1117 < _map1114.size; ++_i1117)
+                  org.apache.thrift.protocol.TMap _map1138 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Materialization>(2*_map1138.size);
+                  String _key1139;
+                  Materialization _val1140;
+                  for (int _i1141 = 0; _i1141 < _map1138.size; ++_i1141)
                   {
-                    _key1115 = iprot.readString();
-                    _val1116 = new Materialization();
-                    _val1116.read(iprot);
-                    struct.success.put(_key1115, _val1116);
+                    _key1139 = iprot.readString();
+                    _val1140 = new Materialization();
+                    _val1140.read(iprot);
+                    struct.success.put(_key1139, _val1140);
                   }
                   iprot.readMapEnd();
                 }
@@ -75971,10 +76103,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Materialization> _iter1118 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1142 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1118.getKey());
-              _iter1118.getValue().write(oprot);
+              oprot.writeString(_iter1142.getKey());
+              _iter1142.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -76029,10 +76161,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Materialization> _iter1119 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1143 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1119.getKey());
-              _iter1119.getValue().write(oprot);
+              oprot.writeString(_iter1143.getKey());
+              _iter1143.getValue().write(oprot);
             }
           }
         }
@@ -76053,16 +76185,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1120 = 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,Materialization>(2*_map1120.size);
-            String _key1121;
-            Materialization _val1122;
-            for (int _i1123 = 0; _i1123 < _map1120.size; ++_i1123)
+            org.apache.thrift.protocol.TMap _map1144 = 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,Materialization>(2*_map1144.size);
+            String _key1145;
+            Materialization _val1146;
+            for (int _i1147 = 0; _i1147 < _map1144.size; ++_i1147)
             {
-              _key1121 = iprot.readString();
-              _val1122 = new Materialization();
-              _val1122.read(iprot);
-              struct.success.put(_key1121, _val1122);
+              _key1145 = iprot.readString();
+              _val1146 = new Materialization();
+              _val1146.read(iprot);
+              struct.success.put(_key1145, _val1146);
             }
           }
           struct.setSuccessIsSet(true);
@@ -78455,13 +78587,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1124 = iprot.readListBegin();
-

<TRUNCATED>

[08/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index eda462e..4467479 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list848.size);
-                WMPool _elem849;
-                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
+                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list872.size);
+                WMPool _elem873;
+                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
                 {
-                  _elem849 = new WMPool();
-                  _elem849.read(iprot);
-                  struct.pools.add(_elem849);
+                  _elem873 = new WMPool();
+                  _elem873.read(iprot);
+                  struct.pools.add(_elem873);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list851 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list851.size);
-                WMMapping _elem852;
-                for (int _i853 = 0; _i853 < _list851.size; ++_i853)
+                org.apache.thrift.protocol.TList _list875 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list875.size);
+                WMMapping _elem876;
+                for (int _i877 = 0; _i877 < _list875.size; ++_i877)
                 {
-                  _elem852 = new WMMapping();
-                  _elem852.read(iprot);
-                  struct.mappings.add(_elem852);
+                  _elem876 = new WMMapping();
+                  _elem876.read(iprot);
+                  struct.mappings.add(_elem876);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list854 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list854.size);
-                WMTrigger _elem855;
-                for (int _i856 = 0; _i856 < _list854.size; ++_i856)
+                org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list878.size);
+                WMTrigger _elem879;
+                for (int _i880 = 0; _i880 < _list878.size; ++_i880)
                 {
-                  _elem855 = new WMTrigger();
-                  _elem855.read(iprot);
-                  struct.triggers.add(_elem855);
+                  _elem879 = new WMTrigger();
+                  _elem879.read(iprot);
+                  struct.triggers.add(_elem879);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list857 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list857.size);
-                WMPoolTrigger _elem858;
-                for (int _i859 = 0; _i859 < _list857.size; ++_i859)
+                org.apache.thrift.protocol.TList _list881 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list881.size);
+                WMPoolTrigger _elem882;
+                for (int _i883 = 0; _i883 < _list881.size; ++_i883)
                 {
-                  _elem858 = new WMPoolTrigger();
-                  _elem858.read(iprot);
-                  struct.poolTriggers.add(_elem858);
+                  _elem882 = new WMPoolTrigger();
+                  _elem882.read(iprot);
+                  struct.poolTriggers.add(_elem882);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter860 : struct.pools)
+          for (WMPool _iter884 : struct.pools)
           {
-            _iter860.write(oprot);
+            _iter884.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter861 : struct.mappings)
+            for (WMMapping _iter885 : struct.mappings)
             {
-              _iter861.write(oprot);
+              _iter885.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter862 : struct.triggers)
+            for (WMTrigger _iter886 : struct.triggers)
             {
-              _iter862.write(oprot);
+              _iter886.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter863 : struct.poolTriggers)
+            for (WMPoolTrigger _iter887 : struct.poolTriggers)
             {
-              _iter863.write(oprot);
+              _iter887.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter864 : struct.pools)
+        for (WMPool _iter888 : struct.pools)
         {
-          _iter864.write(oprot);
+          _iter888.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter865 : struct.mappings)
+          for (WMMapping _iter889 : struct.mappings)
           {
-            _iter865.write(oprot);
+            _iter889.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter866 : struct.triggers)
+          for (WMTrigger _iter890 : struct.triggers)
           {
-            _iter866.write(oprot);
+            _iter890.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter867 : struct.poolTriggers)
+          for (WMPoolTrigger _iter891 : struct.poolTriggers)
           {
-            _iter867.write(oprot);
+            _iter891.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list868 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list868.size);
-        WMPool _elem869;
-        for (int _i870 = 0; _i870 < _list868.size; ++_i870)
+        org.apache.thrift.protocol.TList _list892 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list892.size);
+        WMPool _elem893;
+        for (int _i894 = 0; _i894 < _list892.size; ++_i894)
         {
-          _elem869 = new WMPool();
-          _elem869.read(iprot);
-          struct.pools.add(_elem869);
+          _elem893 = new WMPool();
+          _elem893.read(iprot);
+          struct.pools.add(_elem893);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list871.size);
-          WMMapping _elem872;
-          for (int _i873 = 0; _i873 < _list871.size; ++_i873)
+          org.apache.thrift.protocol.TList _list895 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list895.size);
+          WMMapping _elem896;
+          for (int _i897 = 0; _i897 < _list895.size; ++_i897)
           {
-            _elem872 = new WMMapping();
-            _elem872.read(iprot);
-            struct.mappings.add(_elem872);
+            _elem896 = new WMMapping();
+            _elem896.read(iprot);
+            struct.mappings.add(_elem896);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list874 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list874.size);
-          WMTrigger _elem875;
-          for (int _i876 = 0; _i876 < _list874.size; ++_i876)
+          org.apache.thrift.protocol.TList _list898 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list898.size);
+          WMTrigger _elem899;
+          for (int _i900 = 0; _i900 < _list898.size; ++_i900)
           {
-            _elem875 = new WMTrigger();
-            _elem875.read(iprot);
-            struct.triggers.add(_elem875);
+            _elem899 = new WMTrigger();
+            _elem899.read(iprot);
+            struct.triggers.add(_elem899);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list877.size);
-          WMPoolTrigger _elem878;
-          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
+          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list901.size);
+          WMPoolTrigger _elem902;
+          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
           {
-            _elem878 = new WMPoolTrigger();
-            _elem878.read(iprot);
-            struct.poolTriggers.add(_elem878);
+            _elem902 = new WMPoolTrigger();
+            _elem902.read(iprot);
+            struct.poolTriggers.add(_elem902);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index 9bbc97b..c6cb845 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list880.size);
-                WMResourcePlan _elem881;
-                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
+                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list904.size);
+                WMResourcePlan _elem905;
+                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
                 {
-                  _elem881 = new WMResourcePlan();
-                  _elem881.read(iprot);
-                  struct.resourcePlans.add(_elem881);
+                  _elem905 = new WMResourcePlan();
+                  _elem905.read(iprot);
+                  struct.resourcePlans.add(_elem905);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter883 : struct.resourcePlans)
+            for (WMResourcePlan _iter907 : struct.resourcePlans)
             {
-              _iter883.write(oprot);
+              _iter907.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter884 : struct.resourcePlans)
+          for (WMResourcePlan _iter908 : struct.resourcePlans)
           {
-            _iter884.write(oprot);
+            _iter908.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list885.size);
-          WMResourcePlan _elem886;
-          for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list909.size);
+          WMResourcePlan _elem910;
+          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
           {
-            _elem886 = new WMResourcePlan();
-            _elem886.read(iprot);
-            struct.resourcePlans.add(_elem886);
+            _elem910 = new WMResourcePlan();
+            _elem910.read(iprot);
+            struct.resourcePlans.add(_elem910);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 6918953..9eed335 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list904.size);
-                WMTrigger _elem905;
-                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
+                org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list928.size);
+                WMTrigger _elem929;
+                for (int _i930 = 0; _i930 < _list928.size; ++_i930)
                 {
-                  _elem905 = new WMTrigger();
-                  _elem905.read(iprot);
-                  struct.triggers.add(_elem905);
+                  _elem929 = new WMTrigger();
+                  _elem929.read(iprot);
+                  struct.triggers.add(_elem929);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter907 : struct.triggers)
+            for (WMTrigger _iter931 : struct.triggers)
             {
-              _iter907.write(oprot);
+              _iter931.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter908 : struct.triggers)
+          for (WMTrigger _iter932 : struct.triggers)
           {
-            _iter908.write(oprot);
+            _iter932.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list909.size);
-          WMTrigger _elem910;
-          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
+          org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list933.size);
+          WMTrigger _elem934;
+          for (int _i935 = 0; _i935 < _list933.size; ++_i935)
           {
-            _elem910 = new WMTrigger();
-            _elem910.read(iprot);
-            struct.triggers.add(_elem910);
+            _elem934 = new WMTrigger();
+            _elem934.read(iprot);
+            struct.triggers.add(_elem934);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index 66a478d..ee9251c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list888.size);
-                String _elem889;
-                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list912.size);
+                String _elem913;
+                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
                 {
-                  _elem889 = iprot.readString();
-                  struct.errors.add(_elem889);
+                  _elem913 = iprot.readString();
+                  struct.errors.add(_elem913);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list891 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list891.size);
-                String _elem892;
-                for (int _i893 = 0; _i893 < _list891.size; ++_i893)
+                org.apache.thrift.protocol.TList _list915 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list915.size);
+                String _elem916;
+                for (int _i917 = 0; _i917 < _list915.size; ++_i917)
                 {
-                  _elem892 = iprot.readString();
-                  struct.warnings.add(_elem892);
+                  _elem916 = iprot.readString();
+                  struct.warnings.add(_elem916);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter894 : struct.errors)
+            for (String _iter918 : struct.errors)
             {
-              oprot.writeString(_iter894);
+              oprot.writeString(_iter918);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter895 : struct.warnings)
+            for (String _iter919 : struct.warnings)
             {
-              oprot.writeString(_iter895);
+              oprot.writeString(_iter919);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter896 : struct.errors)
+          for (String _iter920 : struct.errors)
           {
-            oprot.writeString(_iter896);
+            oprot.writeString(_iter920);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter897 : struct.warnings)
+          for (String _iter921 : struct.warnings)
           {
-            oprot.writeString(_iter897);
+            oprot.writeString(_iter921);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list898 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list898.size);
-          String _elem899;
-          for (int _i900 = 0; _i900 < _list898.size; ++_i900)
+          org.apache.thrift.protocol.TList _list922 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list922.size);
+          String _elem923;
+          for (int _i924 = 0; _i924 < _list922.size; ++_i924)
           {
-            _elem899 = iprot.readString();
-            struct.errors.add(_elem899);
+            _elem923 = iprot.readString();
+            struct.errors.add(_elem923);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list901.size);
-          String _elem902;
-          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
+          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list925.size);
+          String _elem926;
+          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
           {
-            _elem902 = iprot.readString();
-            struct.warnings.add(_elem902);
+            _elem926 = iprot.readString();
+            struct.warnings.add(_elem926);
           }
         }
         struct.setWarningsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
new file mode 100644
index 0000000..22f2609
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
@@ -0,0 +1,1012 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteEventInfo implements org.apache.thrift.TBase<WriteEventInfo, WriteEventInfo._Fields>, java.io.Serializable, Cloneable, Comparable<WriteEventInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteEventInfo");
+
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField DATABASE_FIELD_DESC = new org.apache.thrift.protocol.TField("database", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("partition", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField TABLE_OBJ_FIELD_DESC = new org.apache.thrift.protocol.TField("tableObj", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField PARTITION_OBJ_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionObj", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteEventInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteEventInfoTupleSchemeFactory());
+  }
+
+  private long writeId; // required
+  private String database; // required
+  private String table; // required
+  private String files; // required
+  private String partition; // optional
+  private String tableObj; // optional
+  private String partitionObj; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WRITE_ID((short)1, "writeId"),
+    DATABASE((short)2, "database"),
+    TABLE((short)3, "table"),
+    FILES((short)4, "files"),
+    PARTITION((short)5, "partition"),
+    TABLE_OBJ((short)6, "tableObj"),
+    PARTITION_OBJ((short)7, "partitionObj");
+
+    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: // WRITE_ID
+          return WRITE_ID;
+        case 2: // DATABASE
+          return DATABASE;
+        case 3: // TABLE
+          return TABLE;
+        case 4: // FILES
+          return FILES;
+        case 5: // PARTITION
+          return PARTITION;
+        case 6: // TABLE_OBJ
+          return TABLE_OBJ;
+        case 7: // PARTITION_OBJ
+          return PARTITION_OBJ;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __WRITEID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PARTITION,_Fields.TABLE_OBJ,_Fields.PARTITION_OBJ};
+  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.WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("writeId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.DATABASE, new org.apache.thrift.meta_data.FieldMetaData("database", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PARTITION, new org.apache.thrift.meta_data.FieldMetaData("partition", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_OBJ, new org.apache.thrift.meta_data.FieldMetaData("tableObj", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PARTITION_OBJ, new org.apache.thrift.meta_data.FieldMetaData("partitionObj", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteEventInfo.class, metaDataMap);
+  }
+
+  public WriteEventInfo() {
+  }
+
+  public WriteEventInfo(
+    long writeId,
+    String database,
+    String table,
+    String files)
+  {
+    this();
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+    this.database = database;
+    this.table = table;
+    this.files = files;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteEventInfo(WriteEventInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.writeId = other.writeId;
+    if (other.isSetDatabase()) {
+      this.database = other.database;
+    }
+    if (other.isSetTable()) {
+      this.table = other.table;
+    }
+    if (other.isSetFiles()) {
+      this.files = other.files;
+    }
+    if (other.isSetPartition()) {
+      this.partition = other.partition;
+    }
+    if (other.isSetTableObj()) {
+      this.tableObj = other.tableObj;
+    }
+    if (other.isSetPartitionObj()) {
+      this.partitionObj = other.partitionObj;
+    }
+  }
+
+  public WriteEventInfo deepCopy() {
+    return new WriteEventInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    setWriteIdIsSet(false);
+    this.writeId = 0;
+    this.database = null;
+    this.table = null;
+    this.files = null;
+    this.partition = null;
+    this.tableObj = null;
+    this.partitionObj = null;
+  }
+
+  public long getWriteId() {
+    return this.writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+  }
+
+  public void unsetWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field writeId is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  public void setWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEID_ISSET_ID, value);
+  }
+
+  public String getDatabase() {
+    return this.database;
+  }
+
+  public void setDatabase(String database) {
+    this.database = database;
+  }
+
+  public void unsetDatabase() {
+    this.database = null;
+  }
+
+  /** Returns true if field database is set (has been assigned a value) and false otherwise */
+  public boolean isSetDatabase() {
+    return this.database != null;
+  }
+
+  public void setDatabaseIsSet(boolean value) {
+    if (!value) {
+      this.database = null;
+    }
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public void unsetTable() {
+    this.table = null;
+  }
+
+  /** Returns true if field table is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable() {
+    return this.table != null;
+  }
+
+  public void setTableIsSet(boolean value) {
+    if (!value) {
+      this.table = null;
+    }
+  }
+
+  public String getFiles() {
+    return this.files;
+  }
+
+  public void setFiles(String files) {
+    this.files = files;
+  }
+
+  public void unsetFiles() {
+    this.files = null;
+  }
+
+  /** Returns true if field files is set (has been assigned a value) and false otherwise */
+  public boolean isSetFiles() {
+    return this.files != null;
+  }
+
+  public void setFilesIsSet(boolean value) {
+    if (!value) {
+      this.files = null;
+    }
+  }
+
+  public String getPartition() {
+    return this.partition;
+  }
+
+  public void setPartition(String partition) {
+    this.partition = partition;
+  }
+
+  public void unsetPartition() {
+    this.partition = null;
+  }
+
+  /** Returns true if field partition is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartition() {
+    return this.partition != null;
+  }
+
+  public void setPartitionIsSet(boolean value) {
+    if (!value) {
+      this.partition = null;
+    }
+  }
+
+  public String getTableObj() {
+    return this.tableObj;
+  }
+
+  public void setTableObj(String tableObj) {
+    this.tableObj = tableObj;
+  }
+
+  public void unsetTableObj() {
+    this.tableObj = null;
+  }
+
+  /** Returns true if field tableObj is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableObj() {
+    return this.tableObj != null;
+  }
+
+  public void setTableObjIsSet(boolean value) {
+    if (!value) {
+      this.tableObj = null;
+    }
+  }
+
+  public String getPartitionObj() {
+    return this.partitionObj;
+  }
+
+  public void setPartitionObj(String partitionObj) {
+    this.partitionObj = partitionObj;
+  }
+
+  public void unsetPartitionObj() {
+    this.partitionObj = null;
+  }
+
+  /** Returns true if field partitionObj is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitionObj() {
+    return this.partitionObj != null;
+  }
+
+  public void setPartitionObjIsSet(boolean value) {
+    if (!value) {
+      this.partitionObj = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
+    case DATABASE:
+      if (value == null) {
+        unsetDatabase();
+      } else {
+        setDatabase((String)value);
+      }
+      break;
+
+    case TABLE:
+      if (value == null) {
+        unsetTable();
+      } else {
+        setTable((String)value);
+      }
+      break;
+
+    case FILES:
+      if (value == null) {
+        unsetFiles();
+      } else {
+        setFiles((String)value);
+      }
+      break;
+
+    case PARTITION:
+      if (value == null) {
+        unsetPartition();
+      } else {
+        setPartition((String)value);
+      }
+      break;
+
+    case TABLE_OBJ:
+      if (value == null) {
+        unsetTableObj();
+      } else {
+        setTableObj((String)value);
+      }
+      break;
+
+    case PARTITION_OBJ:
+      if (value == null) {
+        unsetPartitionObj();
+      } else {
+        setPartitionObj((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WRITE_ID:
+      return getWriteId();
+
+    case DATABASE:
+      return getDatabase();
+
+    case TABLE:
+      return getTable();
+
+    case FILES:
+      return getFiles();
+
+    case PARTITION:
+      return getPartition();
+
+    case TABLE_OBJ:
+      return getTableObj();
+
+    case PARTITION_OBJ:
+      return getPartitionObj();
+
+    }
+    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 WRITE_ID:
+      return isSetWriteId();
+    case DATABASE:
+      return isSetDatabase();
+    case TABLE:
+      return isSetTable();
+    case FILES:
+      return isSetFiles();
+    case PARTITION:
+      return isSetPartition();
+    case TABLE_OBJ:
+      return isSetTableObj();
+    case PARTITION_OBJ:
+      return isSetPartitionObj();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteEventInfo)
+      return this.equals((WriteEventInfo)that);
+    return false;
+  }
+
+  public boolean equals(WriteEventInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_writeId = true;
+    boolean that_present_writeId = true;
+    if (this_present_writeId || that_present_writeId) {
+      if (!(this_present_writeId && that_present_writeId))
+        return false;
+      if (this.writeId != that.writeId)
+        return false;
+    }
+
+    boolean this_present_database = true && this.isSetDatabase();
+    boolean that_present_database = true && that.isSetDatabase();
+    if (this_present_database || that_present_database) {
+      if (!(this_present_database && that_present_database))
+        return false;
+      if (!this.database.equals(that.database))
+        return false;
+    }
+
+    boolean this_present_table = true && this.isSetTable();
+    boolean that_present_table = true && that.isSetTable();
+    if (this_present_table || that_present_table) {
+      if (!(this_present_table && that_present_table))
+        return false;
+      if (!this.table.equals(that.table))
+        return false;
+    }
+
+    boolean this_present_files = true && this.isSetFiles();
+    boolean that_present_files = true && that.isSetFiles();
+    if (this_present_files || that_present_files) {
+      if (!(this_present_files && that_present_files))
+        return false;
+      if (!this.files.equals(that.files))
+        return false;
+    }
+
+    boolean this_present_partition = true && this.isSetPartition();
+    boolean that_present_partition = true && that.isSetPartition();
+    if (this_present_partition || that_present_partition) {
+      if (!(this_present_partition && that_present_partition))
+        return false;
+      if (!this.partition.equals(that.partition))
+        return false;
+    }
+
+    boolean this_present_tableObj = true && this.isSetTableObj();
+    boolean that_present_tableObj = true && that.isSetTableObj();
+    if (this_present_tableObj || that_present_tableObj) {
+      if (!(this_present_tableObj && that_present_tableObj))
+        return false;
+      if (!this.tableObj.equals(that.tableObj))
+        return false;
+    }
+
+    boolean this_present_partitionObj = true && this.isSetPartitionObj();
+    boolean that_present_partitionObj = true && that.isSetPartitionObj();
+    if (this_present_partitionObj || that_present_partitionObj) {
+      if (!(this_present_partitionObj && that_present_partitionObj))
+        return false;
+      if (!this.partitionObj.equals(that.partitionObj))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_writeId = true;
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
+    boolean present_database = true && (isSetDatabase());
+    list.add(present_database);
+    if (present_database)
+      list.add(database);
+
+    boolean present_table = true && (isSetTable());
+    list.add(present_table);
+    if (present_table)
+      list.add(table);
+
+    boolean present_files = true && (isSetFiles());
+    list.add(present_files);
+    if (present_files)
+      list.add(files);
+
+    boolean present_partition = true && (isSetPartition());
+    list.add(present_partition);
+    if (present_partition)
+      list.add(partition);
+
+    boolean present_tableObj = true && (isSetTableObj());
+    list.add(present_tableObj);
+    if (present_tableObj)
+      list.add(tableObj);
+
+    boolean present_partitionObj = true && (isSetPartitionObj());
+    list.add(present_partitionObj);
+    if (present_partitionObj)
+      list.add(partitionObj);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteEventInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetWriteId()).compareTo(other.isSetWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeId, other.writeId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDatabase()).compareTo(other.isSetDatabase());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDatabase()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.database, other.database);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFiles()).compareTo(other.isSetFiles());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFiles()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartition()).compareTo(other.isSetPartition());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartition()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partition, other.partition);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableObj()).compareTo(other.isSetTableObj());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableObj()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableObj, other.tableObj);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartitionObj()).compareTo(other.isSetPartitionObj());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitionObj()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionObj, other.partitionObj);
+      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("WriteEventInfo(");
+    boolean first = true;
+
+    sb.append("writeId:");
+    sb.append(this.writeId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("database:");
+    if (this.database == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.database);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table:");
+    if (this.table == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("files:");
+    if (this.files == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.files);
+    }
+    first = false;
+    if (isSetPartition()) {
+      if (!first) sb.append(", ");
+      sb.append("partition:");
+      if (this.partition == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partition);
+      }
+      first = false;
+    }
+    if (isSetTableObj()) {
+      if (!first) sb.append(", ");
+      sb.append("tableObj:");
+      if (this.tableObj == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableObj);
+      }
+      first = false;
+    }
+    if (isSetPartitionObj()) {
+      if (!first) sb.append(", ");
+      sb.append("partitionObj:");
+      if (this.partitionObj == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partitionObj);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'writeId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetDatabase()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'database' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'table' is unset! Struct:" + toString());
+    }
+
+    if (!isSetFiles()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'files' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WriteEventInfoStandardSchemeFactory implements SchemeFactory {
+    public WriteEventInfoStandardScheme getScheme() {
+      return new WriteEventInfoStandardScheme();
+    }
+  }
+
+  private static class WriteEventInfoStandardScheme extends StandardScheme<WriteEventInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteEventInfo 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: // WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.writeId = iprot.readI64();
+              struct.setWriteIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DATABASE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.database = iprot.readString();
+              struct.setDatabaseIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table = iprot.readString();
+              struct.setTableIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // FILES
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.files = iprot.readString();
+              struct.setFilesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // PARTITION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.partition = iprot.readString();
+              struct.setPartitionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // TABLE_OBJ
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableObj = iprot.readString();
+              struct.setTableObjIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // PARTITION_OBJ
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.partitionObj = iprot.readString();
+              struct.setPartitionObjIsSet(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, WriteEventInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.writeId);
+      oprot.writeFieldEnd();
+      if (struct.database != null) {
+        oprot.writeFieldBegin(DATABASE_FIELD_DESC);
+        oprot.writeString(struct.database);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table != null) {
+        oprot.writeFieldBegin(TABLE_FIELD_DESC);
+        oprot.writeString(struct.table);
+        oprot.writeFieldEnd();
+      }
+      if (struct.files != null) {
+        oprot.writeFieldBegin(FILES_FIELD_DESC);
+        oprot.writeString(struct.files);
+        oprot.writeFieldEnd();
+      }
+      if (struct.partition != null) {
+        if (struct.isSetPartition()) {
+          oprot.writeFieldBegin(PARTITION_FIELD_DESC);
+          oprot.writeString(struct.partition);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.tableObj != null) {
+        if (struct.isSetTableObj()) {
+          oprot.writeFieldBegin(TABLE_OBJ_FIELD_DESC);
+          oprot.writeString(struct.tableObj);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.partitionObj != null) {
+        if (struct.isSetPartitionObj()) {
+          oprot.writeFieldBegin(PARTITION_OBJ_FIELD_DESC);
+          oprot.writeString(struct.partitionObj);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteEventInfoTupleSchemeFactory implements SchemeFactory {
+    public WriteEventInfoTupleScheme getScheme() {
+      return new WriteEventInfoTupleScheme();
+    }
+  }
+
+  private static class WriteEventInfoTupleScheme extends TupleScheme<WriteEventInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.writeId);
+      oprot.writeString(struct.database);
+      oprot.writeString(struct.table);
+      oprot.writeString(struct.files);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartition()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTableObj()) {
+        optionals.set(1);
+      }
+      if (struct.isSetPartitionObj()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetPartition()) {
+        oprot.writeString(struct.partition);
+      }
+      if (struct.isSetTableObj()) {
+        oprot.writeString(struct.tableObj);
+      }
+      if (struct.isSetPartitionObj()) {
+        oprot.writeString(struct.partitionObj);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.writeId = iprot.readI64();
+      struct.setWriteIdIsSet(true);
+      struct.database = iprot.readString();
+      struct.setDatabaseIsSet(true);
+      struct.table = iprot.readString();
+      struct.setTableIsSet(true);
+      struct.files = iprot.readString();
+      struct.setFilesIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.partition = iprot.readString();
+        struct.setPartitionIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.tableObj = iprot.readString();
+        struct.setTableObjIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.partitionObj = iprot.readString();
+        struct.setPartitionObjIsSet(true);
+      }
+    }
+  }
+
+}
+


[12/16] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index bc4d168..7f06b3b 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -16592,6 +16592,11 @@ void CommitTxnRequest::__set_replPolicy(const std::string& val) {
 __isset.replPolicy = true;
 }
 
+void CommitTxnRequest::__set_writeEventInfos(const std::vector<WriteEventInfo> & val) {
+  this->writeEventInfos = val;
+__isset.writeEventInfos = true;
+}
+
 uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -16630,6 +16635,26 @@ uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->writeEventInfos.clear();
+            uint32_t _size673;
+            ::apache::thrift::protocol::TType _etype676;
+            xfer += iprot->readListBegin(_etype676, _size673);
+            this->writeEventInfos.resize(_size673);
+            uint32_t _i677;
+            for (_i677 = 0; _i677 < _size673; ++_i677)
+            {
+              xfer += this->writeEventInfos[_i677].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.writeEventInfos = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -16658,6 +16683,19 @@ uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeString(this->replPolicy);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeEventInfos) {
+    xfer += oprot->writeFieldBegin("writeEventInfos", ::apache::thrift::protocol::T_LIST, 3);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->writeEventInfos.size()));
+      std::vector<WriteEventInfo> ::const_iterator _iter678;
+      for (_iter678 = this->writeEventInfos.begin(); _iter678 != this->writeEventInfos.end(); ++_iter678)
+      {
+        xfer += (*_iter678).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -16667,18 +16705,21 @@ void swap(CommitTxnRequest &a, CommitTxnRequest &b) {
   using ::std::swap;
   swap(a.txnid, b.txnid);
   swap(a.replPolicy, b.replPolicy);
+  swap(a.writeEventInfos, b.writeEventInfos);
   swap(a.__isset, b.__isset);
 }
 
-CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other673) {
-  txnid = other673.txnid;
-  replPolicy = other673.replPolicy;
-  __isset = other673.__isset;
+CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other679) {
+  txnid = other679.txnid;
+  replPolicy = other679.replPolicy;
+  writeEventInfos = other679.writeEventInfos;
+  __isset = other679.__isset;
 }
-CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other674) {
-  txnid = other674.txnid;
-  replPolicy = other674.replPolicy;
-  __isset = other674.__isset;
+CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other680) {
+  txnid = other680.txnid;
+  replPolicy = other680.replPolicy;
+  writeEventInfos = other680.writeEventInfos;
+  __isset = other680.__isset;
   return *this;
 }
 void CommitTxnRequest::printTo(std::ostream& out) const {
@@ -16686,6 +16727,231 @@ void CommitTxnRequest::printTo(std::ostream& out) const {
   out << "CommitTxnRequest(";
   out << "txnid=" << to_string(txnid);
   out << ", " << "replPolicy="; (__isset.replPolicy ? (out << to_string(replPolicy)) : (out << "<null>"));
+  out << ", " << "writeEventInfos="; (__isset.writeEventInfos ? (out << to_string(writeEventInfos)) : (out << "<null>"));
+  out << ")";
+}
+
+
+WriteEventInfo::~WriteEventInfo() throw() {
+}
+
+
+void WriteEventInfo::__set_writeId(const int64_t val) {
+  this->writeId = val;
+}
+
+void WriteEventInfo::__set_database(const std::string& val) {
+  this->database = val;
+}
+
+void WriteEventInfo::__set_table(const std::string& val) {
+  this->table = val;
+}
+
+void WriteEventInfo::__set_files(const std::string& val) {
+  this->files = val;
+}
+
+void WriteEventInfo::__set_partition(const std::string& val) {
+  this->partition = val;
+__isset.partition = true;
+}
+
+void WriteEventInfo::__set_tableObj(const std::string& val) {
+  this->tableObj = val;
+__isset.tableObj = true;
+}
+
+void WriteEventInfo::__set_partitionObj(const std::string& val) {
+  this->partitionObj = val;
+__isset.partitionObj = true;
+}
+
+uint32_t WriteEventInfo::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_writeId = false;
+  bool isset_database = false;
+  bool isset_table = false;
+  bool isset_files = 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_I64) {
+          xfer += iprot->readI64(this->writeId);
+          isset_writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->database);
+          isset_database = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->table);
+          isset_table = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->files);
+          isset_files = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->partition);
+          this->__isset.partition = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableObj);
+          this->__isset.tableObj = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->partitionObj);
+          this->__isset.partitionObj = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_writeId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_database)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_table)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_files)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t WriteEventInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("WriteEventInfo");
+
+  xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64(this->writeId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->database);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->table);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("files", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeString(this->files);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.partition) {
+    xfer += oprot->writeFieldBegin("partition", ::apache::thrift::protocol::T_STRING, 5);
+    xfer += oprot->writeString(this->partition);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.tableObj) {
+    xfer += oprot->writeFieldBegin("tableObj", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeString(this->tableObj);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.partitionObj) {
+    xfer += oprot->writeFieldBegin("partitionObj", ::apache::thrift::protocol::T_STRING, 7);
+    xfer += oprot->writeString(this->partitionObj);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(WriteEventInfo &a, WriteEventInfo &b) {
+  using ::std::swap;
+  swap(a.writeId, b.writeId);
+  swap(a.database, b.database);
+  swap(a.table, b.table);
+  swap(a.files, b.files);
+  swap(a.partition, b.partition);
+  swap(a.tableObj, b.tableObj);
+  swap(a.partitionObj, b.partitionObj);
+  swap(a.__isset, b.__isset);
+}
+
+WriteEventInfo::WriteEventInfo(const WriteEventInfo& other681) {
+  writeId = other681.writeId;
+  database = other681.database;
+  table = other681.table;
+  files = other681.files;
+  partition = other681.partition;
+  tableObj = other681.tableObj;
+  partitionObj = other681.partitionObj;
+  __isset = other681.__isset;
+}
+WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other682) {
+  writeId = other682.writeId;
+  database = other682.database;
+  table = other682.table;
+  files = other682.files;
+  partition = other682.partition;
+  tableObj = other682.tableObj;
+  partitionObj = other682.partitionObj;
+  __isset = other682.__isset;
+  return *this;
+}
+void WriteEventInfo::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "WriteEventInfo(";
+  out << "writeId=" << to_string(writeId);
+  out << ", " << "database=" << to_string(database);
+  out << ", " << "table=" << to_string(table);
+  out << ", " << "files=" << to_string(files);
+  out << ", " << "partition="; (__isset.partition ? (out << to_string(partition)) : (out << "<null>"));
+  out << ", " << "tableObj="; (__isset.tableObj ? (out << to_string(tableObj)) : (out << "<null>"));
+  out << ", " << "partitionObj="; (__isset.partitionObj ? (out << to_string(partitionObj)) : (out << "<null>"));
   out << ")";
 }
 
@@ -16789,14 +17055,14 @@ uint32_t ReplTblWriteIdStateRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size675;
-            ::apache::thrift::protocol::TType _etype678;
-            xfer += iprot->readListBegin(_etype678, _size675);
-            this->partNames.resize(_size675);
-            uint32_t _i679;
-            for (_i679 = 0; _i679 < _size675; ++_i679)
+            uint32_t _size683;
+            ::apache::thrift::protocol::TType _etype686;
+            xfer += iprot->readListBegin(_etype686, _size683);
+            this->partNames.resize(_size683);
+            uint32_t _i687;
+            for (_i687 = 0; _i687 < _size683; ++_i687)
             {
-              xfer += iprot->readString(this->partNames[_i679]);
+              xfer += iprot->readString(this->partNames[_i687]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16856,10 +17122,10 @@ uint32_t ReplTblWriteIdStateRequest::write(::apache::thrift::protocol::TProtocol
     xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 6);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-      std::vector<std::string> ::const_iterator _iter680;
-      for (_iter680 = this->partNames.begin(); _iter680 != this->partNames.end(); ++_iter680)
+      std::vector<std::string> ::const_iterator _iter688;
+      for (_iter688 = this->partNames.begin(); _iter688 != this->partNames.end(); ++_iter688)
       {
-        xfer += oprot->writeString((*_iter680));
+        xfer += oprot->writeString((*_iter688));
       }
       xfer += oprot->writeListEnd();
     }
@@ -16881,23 +17147,23 @@ void swap(ReplTblWriteIdStateRequest &a, ReplTblWriteIdStateRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other681) {
-  validWriteIdlist = other681.validWriteIdlist;
-  user = other681.user;
-  hostName = other681.hostName;
-  dbName = other681.dbName;
-  tableName = other681.tableName;
-  partNames = other681.partNames;
-  __isset = other681.__isset;
-}
-ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other682) {
-  validWriteIdlist = other682.validWriteIdlist;
-  user = other682.user;
-  hostName = other682.hostName;
-  dbName = other682.dbName;
-  tableName = other682.tableName;
-  partNames = other682.partNames;
-  __isset = other682.__isset;
+ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other689) {
+  validWriteIdlist = other689.validWriteIdlist;
+  user = other689.user;
+  hostName = other689.hostName;
+  dbName = other689.dbName;
+  tableName = other689.tableName;
+  partNames = other689.partNames;
+  __isset = other689.__isset;
+}
+ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other690) {
+  validWriteIdlist = other690.validWriteIdlist;
+  user = other690.user;
+  hostName = other690.hostName;
+  dbName = other690.dbName;
+  tableName = other690.tableName;
+  partNames = other690.partNames;
+  __isset = other690.__isset;
   return *this;
 }
 void ReplTblWriteIdStateRequest::printTo(std::ostream& out) const {
@@ -16952,14 +17218,14 @@ uint32_t GetValidWriteIdsRequest::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fullTableNames.clear();
-            uint32_t _size683;
-            ::apache::thrift::protocol::TType _etype686;
-            xfer += iprot->readListBegin(_etype686, _size683);
-            this->fullTableNames.resize(_size683);
-            uint32_t _i687;
-            for (_i687 = 0; _i687 < _size683; ++_i687)
+            uint32_t _size691;
+            ::apache::thrift::protocol::TType _etype694;
+            xfer += iprot->readListBegin(_etype694, _size691);
+            this->fullTableNames.resize(_size691);
+            uint32_t _i695;
+            for (_i695 = 0; _i695 < _size691; ++_i695)
             {
-              xfer += iprot->readString(this->fullTableNames[_i687]);
+              xfer += iprot->readString(this->fullTableNames[_i695]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17000,10 +17266,10 @@ uint32_t GetValidWriteIdsRequest::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("fullTableNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->fullTableNames.size()));
-    std::vector<std::string> ::const_iterator _iter688;
-    for (_iter688 = this->fullTableNames.begin(); _iter688 != this->fullTableNames.end(); ++_iter688)
+    std::vector<std::string> ::const_iterator _iter696;
+    for (_iter696 = this->fullTableNames.begin(); _iter696 != this->fullTableNames.end(); ++_iter696)
     {
-      xfer += oprot->writeString((*_iter688));
+      xfer += oprot->writeString((*_iter696));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17024,13 +17290,13 @@ void swap(GetValidWriteIdsRequest &a, GetValidWriteIdsRequest &b) {
   swap(a.validTxnList, b.validTxnList);
 }
 
-GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other689) {
-  fullTableNames = other689.fullTableNames;
-  validTxnList = other689.validTxnList;
+GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other697) {
+  fullTableNames = other697.fullTableNames;
+  validTxnList = other697.validTxnList;
 }
-GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other690) {
-  fullTableNames = other690.fullTableNames;
-  validTxnList = other690.validTxnList;
+GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other698) {
+  fullTableNames = other698.fullTableNames;
+  validTxnList = other698.validTxnList;
   return *this;
 }
 void GetValidWriteIdsRequest::printTo(std::ostream& out) const {
@@ -17112,14 +17378,14 @@ uint32_t TableValidWriteIds::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->invalidWriteIds.clear();
-            uint32_t _size691;
-            ::apache::thrift::protocol::TType _etype694;
-            xfer += iprot->readListBegin(_etype694, _size691);
-            this->invalidWriteIds.resize(_size691);
-            uint32_t _i695;
-            for (_i695 = 0; _i695 < _size691; ++_i695)
+            uint32_t _size699;
+            ::apache::thrift::protocol::TType _etype702;
+            xfer += iprot->readListBegin(_etype702, _size699);
+            this->invalidWriteIds.resize(_size699);
+            uint32_t _i703;
+            for (_i703 = 0; _i703 < _size699; ++_i703)
             {
-              xfer += iprot->readI64(this->invalidWriteIds[_i695]);
+              xfer += iprot->readI64(this->invalidWriteIds[_i703]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17180,10 +17446,10 @@ uint32_t TableValidWriteIds::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("invalidWriteIds", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->invalidWriteIds.size()));
-    std::vector<int64_t> ::const_iterator _iter696;
-    for (_iter696 = this->invalidWriteIds.begin(); _iter696 != this->invalidWriteIds.end(); ++_iter696)
+    std::vector<int64_t> ::const_iterator _iter704;
+    for (_iter704 = this->invalidWriteIds.begin(); _iter704 != this->invalidWriteIds.end(); ++_iter704)
     {
-      xfer += oprot->writeI64((*_iter696));
+      xfer += oprot->writeI64((*_iter704));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17213,21 +17479,21 @@ void swap(TableValidWriteIds &a, TableValidWriteIds &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other697) {
-  fullTableName = other697.fullTableName;
-  writeIdHighWaterMark = other697.writeIdHighWaterMark;
-  invalidWriteIds = other697.invalidWriteIds;
-  minOpenWriteId = other697.minOpenWriteId;
-  abortedBits = other697.abortedBits;
-  __isset = other697.__isset;
-}
-TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other698) {
-  fullTableName = other698.fullTableName;
-  writeIdHighWaterMark = other698.writeIdHighWaterMark;
-  invalidWriteIds = other698.invalidWriteIds;
-  minOpenWriteId = other698.minOpenWriteId;
-  abortedBits = other698.abortedBits;
-  __isset = other698.__isset;
+TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other705) {
+  fullTableName = other705.fullTableName;
+  writeIdHighWaterMark = other705.writeIdHighWaterMark;
+  invalidWriteIds = other705.invalidWriteIds;
+  minOpenWriteId = other705.minOpenWriteId;
+  abortedBits = other705.abortedBits;
+  __isset = other705.__isset;
+}
+TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other706) {
+  fullTableName = other706.fullTableName;
+  writeIdHighWaterMark = other706.writeIdHighWaterMark;
+  invalidWriteIds = other706.invalidWriteIds;
+  minOpenWriteId = other706.minOpenWriteId;
+  abortedBits = other706.abortedBits;
+  __isset = other706.__isset;
   return *this;
 }
 void TableValidWriteIds::printTo(std::ostream& out) const {
@@ -17276,14 +17542,14 @@ uint32_t GetValidWriteIdsResponse::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tblValidWriteIds.clear();
-            uint32_t _size699;
-            ::apache::thrift::protocol::TType _etype702;
-            xfer += iprot->readListBegin(_etype702, _size699);
-            this->tblValidWriteIds.resize(_size699);
-            uint32_t _i703;
-            for (_i703 = 0; _i703 < _size699; ++_i703)
+            uint32_t _size707;
+            ::apache::thrift::protocol::TType _etype710;
+            xfer += iprot->readListBegin(_etype710, _size707);
+            this->tblValidWriteIds.resize(_size707);
+            uint32_t _i711;
+            for (_i711 = 0; _i711 < _size707; ++_i711)
             {
-              xfer += this->tblValidWriteIds[_i703].read(iprot);
+              xfer += this->tblValidWriteIds[_i711].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17314,10 +17580,10 @@ uint32_t GetValidWriteIdsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("tblValidWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tblValidWriteIds.size()));
-    std::vector<TableValidWriteIds> ::const_iterator _iter704;
-    for (_iter704 = this->tblValidWriteIds.begin(); _iter704 != this->tblValidWriteIds.end(); ++_iter704)
+    std::vector<TableValidWriteIds> ::const_iterator _iter712;
+    for (_iter712 = this->tblValidWriteIds.begin(); _iter712 != this->tblValidWriteIds.end(); ++_iter712)
     {
-      xfer += (*_iter704).write(oprot);
+      xfer += (*_iter712).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17333,11 +17599,11 @@ void swap(GetValidWriteIdsResponse &a, GetValidWriteIdsResponse &b) {
   swap(a.tblValidWriteIds, b.tblValidWriteIds);
 }
 
-GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other705) {
-  tblValidWriteIds = other705.tblValidWriteIds;
+GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other713) {
+  tblValidWriteIds = other713.tblValidWriteIds;
 }
-GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other706) {
-  tblValidWriteIds = other706.tblValidWriteIds;
+GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other714) {
+  tblValidWriteIds = other714.tblValidWriteIds;
   return *this;
 }
 void GetValidWriteIdsResponse::printTo(std::ostream& out) const {
@@ -17418,14 +17684,14 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnIds.clear();
-            uint32_t _size707;
-            ::apache::thrift::protocol::TType _etype710;
-            xfer += iprot->readListBegin(_etype710, _size707);
-            this->txnIds.resize(_size707);
-            uint32_t _i711;
-            for (_i711 = 0; _i711 < _size707; ++_i711)
+            uint32_t _size715;
+            ::apache::thrift::protocol::TType _etype718;
+            xfer += iprot->readListBegin(_etype718, _size715);
+            this->txnIds.resize(_size715);
+            uint32_t _i719;
+            for (_i719 = 0; _i719 < _size715; ++_i719)
             {
-              xfer += iprot->readI64(this->txnIds[_i711]);
+              xfer += iprot->readI64(this->txnIds[_i719]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17446,14 +17712,14 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->srcTxnToWriteIdList.clear();
-            uint32_t _size712;
-            ::apache::thrift::protocol::TType _etype715;
-            xfer += iprot->readListBegin(_etype715, _size712);
-            this->srcTxnToWriteIdList.resize(_size712);
-            uint32_t _i716;
-            for (_i716 = 0; _i716 < _size712; ++_i716)
+            uint32_t _size720;
+            ::apache::thrift::protocol::TType _etype723;
+            xfer += iprot->readListBegin(_etype723, _size720);
+            this->srcTxnToWriteIdList.resize(_size720);
+            uint32_t _i724;
+            for (_i724 = 0; _i724 < _size720; ++_i724)
             {
-              xfer += this->srcTxnToWriteIdList[_i716].read(iprot);
+              xfer += this->srcTxnToWriteIdList[_i724].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17495,10 +17761,10 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("txnIds", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txnIds.size()));
-      std::vector<int64_t> ::const_iterator _iter717;
-      for (_iter717 = this->txnIds.begin(); _iter717 != this->txnIds.end(); ++_iter717)
+      std::vector<int64_t> ::const_iterator _iter725;
+      for (_iter725 = this->txnIds.begin(); _iter725 != this->txnIds.end(); ++_iter725)
       {
-        xfer += oprot->writeI64((*_iter717));
+        xfer += oprot->writeI64((*_iter725));
       }
       xfer += oprot->writeListEnd();
     }
@@ -17513,10 +17779,10 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("srcTxnToWriteIdList", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->srcTxnToWriteIdList.size()));
-      std::vector<TxnToWriteId> ::const_iterator _iter718;
-      for (_iter718 = this->srcTxnToWriteIdList.begin(); _iter718 != this->srcTxnToWriteIdList.end(); ++_iter718)
+      std::vector<TxnToWriteId> ::const_iterator _iter726;
+      for (_iter726 = this->srcTxnToWriteIdList.begin(); _iter726 != this->srcTxnToWriteIdList.end(); ++_iter726)
       {
-        xfer += (*_iter718).write(oprot);
+        xfer += (*_iter726).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -17537,21 +17803,21 @@ void swap(AllocateTableWriteIdsRequest &a, AllocateTableWriteIdsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other719) {
-  dbName = other719.dbName;
-  tableName = other719.tableName;
-  txnIds = other719.txnIds;
-  replPolicy = other719.replPolicy;
-  srcTxnToWriteIdList = other719.srcTxnToWriteIdList;
-  __isset = other719.__isset;
-}
-AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other720) {
-  dbName = other720.dbName;
-  tableName = other720.tableName;
-  txnIds = other720.txnIds;
-  replPolicy = other720.replPolicy;
-  srcTxnToWriteIdList = other720.srcTxnToWriteIdList;
-  __isset = other720.__isset;
+AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other727) {
+  dbName = other727.dbName;
+  tableName = other727.tableName;
+  txnIds = other727.txnIds;
+  replPolicy = other727.replPolicy;
+  srcTxnToWriteIdList = other727.srcTxnToWriteIdList;
+  __isset = other727.__isset;
+}
+AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other728) {
+  dbName = other728.dbName;
+  tableName = other728.tableName;
+  txnIds = other728.txnIds;
+  replPolicy = other728.replPolicy;
+  srcTxnToWriteIdList = other728.srcTxnToWriteIdList;
+  __isset = other728.__isset;
   return *this;
 }
 void AllocateTableWriteIdsRequest::printTo(std::ostream& out) const {
@@ -17657,13 +17923,13 @@ void swap(TxnToWriteId &a, TxnToWriteId &b) {
   swap(a.writeId, b.writeId);
 }
 
-TxnToWriteId::TxnToWriteId(const TxnToWriteId& other721) {
-  txnId = other721.txnId;
-  writeId = other721.writeId;
+TxnToWriteId::TxnToWriteId(const TxnToWriteId& other729) {
+  txnId = other729.txnId;
+  writeId = other729.writeId;
 }
-TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other722) {
-  txnId = other722.txnId;
-  writeId = other722.writeId;
+TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other730) {
+  txnId = other730.txnId;
+  writeId = other730.writeId;
   return *this;
 }
 void TxnToWriteId::printTo(std::ostream& out) const {
@@ -17709,14 +17975,14 @@ uint32_t AllocateTableWriteIdsResponse::read(::apache::thrift::protocol::TProtoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnToWriteIds.clear();
-            uint32_t _size723;
-            ::apache::thrift::protocol::TType _etype726;
-            xfer += iprot->readListBegin(_etype726, _size723);
-            this->txnToWriteIds.resize(_size723);
-            uint32_t _i727;
-            for (_i727 = 0; _i727 < _size723; ++_i727)
+            uint32_t _size731;
+            ::apache::thrift::protocol::TType _etype734;
+            xfer += iprot->readListBegin(_etype734, _size731);
+            this->txnToWriteIds.resize(_size731);
+            uint32_t _i735;
+            for (_i735 = 0; _i735 < _size731; ++_i735)
             {
-              xfer += this->txnToWriteIds[_i727].read(iprot);
+              xfer += this->txnToWriteIds[_i735].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17747,10 +18013,10 @@ uint32_t AllocateTableWriteIdsResponse::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("txnToWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->txnToWriteIds.size()));
-    std::vector<TxnToWriteId> ::const_iterator _iter728;
-    for (_iter728 = this->txnToWriteIds.begin(); _iter728 != this->txnToWriteIds.end(); ++_iter728)
+    std::vector<TxnToWriteId> ::const_iterator _iter736;
+    for (_iter736 = this->txnToWriteIds.begin(); _iter736 != this->txnToWriteIds.end(); ++_iter736)
     {
-      xfer += (*_iter728).write(oprot);
+      xfer += (*_iter736).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17766,11 +18032,11 @@ void swap(AllocateTableWriteIdsResponse &a, AllocateTableWriteIdsResponse &b) {
   swap(a.txnToWriteIds, b.txnToWriteIds);
 }
 
-AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other729) {
-  txnToWriteIds = other729.txnToWriteIds;
+AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other737) {
+  txnToWriteIds = other737.txnToWriteIds;
 }
-AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other730) {
-  txnToWriteIds = other730.txnToWriteIds;
+AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other738) {
+  txnToWriteIds = other738.txnToWriteIds;
   return *this;
 }
 void AllocateTableWriteIdsResponse::printTo(std::ostream& out) const {
@@ -17848,9 +18114,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast731;
-          xfer += iprot->readI32(ecast731);
-          this->type = (LockType::type)ecast731;
+          int32_t ecast739;
+          xfer += iprot->readI32(ecast739);
+          this->type = (LockType::type)ecast739;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17858,9 +18124,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast732;
-          xfer += iprot->readI32(ecast732);
-          this->level = (LockLevel::type)ecast732;
+          int32_t ecast740;
+          xfer += iprot->readI32(ecast740);
+          this->level = (LockLevel::type)ecast740;
           isset_level = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17892,9 +18158,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast733;
-          xfer += iprot->readI32(ecast733);
-          this->operationType = (DataOperationType::type)ecast733;
+          int32_t ecast741;
+          xfer += iprot->readI32(ecast741);
+          this->operationType = (DataOperationType::type)ecast741;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17994,27 +18260,27 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other734) {
-  type = other734.type;
-  level = other734.level;
-  dbname = other734.dbname;
-  tablename = other734.tablename;
-  partitionname = other734.partitionname;
-  operationType = other734.operationType;
-  isTransactional = other734.isTransactional;
-  isDynamicPartitionWrite = other734.isDynamicPartitionWrite;
-  __isset = other734.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other735) {
-  type = other735.type;
-  level = other735.level;
-  dbname = other735.dbname;
-  tablename = other735.tablename;
-  partitionname = other735.partitionname;
-  operationType = other735.operationType;
-  isTransactional = other735.isTransactional;
-  isDynamicPartitionWrite = other735.isDynamicPartitionWrite;
-  __isset = other735.__isset;
+LockComponent::LockComponent(const LockComponent& other742) {
+  type = other742.type;
+  level = other742.level;
+  dbname = other742.dbname;
+  tablename = other742.tablename;
+  partitionname = other742.partitionname;
+  operationType = other742.operationType;
+  isTransactional = other742.isTransactional;
+  isDynamicPartitionWrite = other742.isDynamicPartitionWrite;
+  __isset = other742.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other743) {
+  type = other743.type;
+  level = other743.level;
+  dbname = other743.dbname;
+  tablename = other743.tablename;
+  partitionname = other743.partitionname;
+  operationType = other743.operationType;
+  isTransactional = other743.isTransactional;
+  isDynamicPartitionWrite = other743.isDynamicPartitionWrite;
+  __isset = other743.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -18086,14 +18352,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _etype739;
-            xfer += iprot->readListBegin(_etype739, _size736);
-            this->component.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->component.resize(_size744);
+            uint32_t _i748;
+            for (_i748 = 0; _i748 < _size744; ++_i748)
             {
-              xfer += this->component[_i740].read(iprot);
+              xfer += this->component[_i748].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18160,10 +18426,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 _iter741;
-    for (_iter741 = this->component.begin(); _iter741 != this->component.end(); ++_iter741)
+    std::vector<LockComponent> ::const_iterator _iter749;
+    for (_iter749 = this->component.begin(); _iter749 != this->component.end(); ++_iter749)
     {
-      xfer += (*_iter741).write(oprot);
+      xfer += (*_iter749).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18202,21 +18468,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other742) {
-  component = other742.component;
-  txnid = other742.txnid;
-  user = other742.user;
-  hostname = other742.hostname;
-  agentInfo = other742.agentInfo;
-  __isset = other742.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other743) {
-  component = other743.component;
-  txnid = other743.txnid;
-  user = other743.user;
-  hostname = other743.hostname;
-  agentInfo = other743.agentInfo;
-  __isset = other743.__isset;
+LockRequest::LockRequest(const LockRequest& other750) {
+  component = other750.component;
+  txnid = other750.txnid;
+  user = other750.user;
+  hostname = other750.hostname;
+  agentInfo = other750.agentInfo;
+  __isset = other750.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other751) {
+  component = other751.component;
+  txnid = other751.txnid;
+  user = other751.user;
+  hostname = other751.hostname;
+  agentInfo = other751.agentInfo;
+  __isset = other751.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -18276,9 +18542,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast744;
-          xfer += iprot->readI32(ecast744);
-          this->state = (LockState::type)ecast744;
+          int32_t ecast752;
+          xfer += iprot->readI32(ecast752);
+          this->state = (LockState::type)ecast752;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18324,13 +18590,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other745) {
-  lockid = other745.lockid;
-  state = other745.state;
+LockResponse::LockResponse(const LockResponse& other753) {
+  lockid = other753.lockid;
+  state = other753.state;
 }
-LockResponse& LockResponse::operator=(const LockResponse& other746) {
-  lockid = other746.lockid;
-  state = other746.state;
+LockResponse& LockResponse::operator=(const LockResponse& other754) {
+  lockid = other754.lockid;
+  state = other754.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -18452,17 +18718,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other747) {
-  lockid = other747.lockid;
-  txnid = other747.txnid;
-  elapsed_ms = other747.elapsed_ms;
-  __isset = other747.__isset;
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other755) {
+  lockid = other755.lockid;
+  txnid = other755.txnid;
+  elapsed_ms = other755.elapsed_ms;
+  __isset = other755.__isset;
 }
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other748) {
-  lockid = other748.lockid;
-  txnid = other748.txnid;
-  elapsed_ms = other748.elapsed_ms;
-  __isset = other748.__isset;
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other756) {
+  lockid = other756.lockid;
+  txnid = other756.txnid;
+  elapsed_ms = other756.elapsed_ms;
+  __isset = other756.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -18546,11 +18812,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other749) {
-  lockid = other749.lockid;
+UnlockRequest::UnlockRequest(const UnlockRequest& other757) {
+  lockid = other757.lockid;
 }
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other750) {
-  lockid = other750.lockid;
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other758) {
+  lockid = other758.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -18689,19 +18955,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other751) {
-  dbname = other751.dbname;
-  tablename = other751.tablename;
-  partname = other751.partname;
-  isExtended = other751.isExtended;
-  __isset = other751.__isset;
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other759) {
+  dbname = other759.dbname;
+  tablename = other759.tablename;
+  partname = other759.partname;
+  isExtended = other759.isExtended;
+  __isset = other759.__isset;
 }
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other752) {
-  dbname = other752.dbname;
-  tablename = other752.tablename;
-  partname = other752.partname;
-  isExtended = other752.isExtended;
-  __isset = other752.__isset;
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other760) {
+  dbname = other760.dbname;
+  tablename = other760.tablename;
+  partname = other760.partname;
+  isExtended = other760.isExtended;
+  __isset = other760.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -18854,9 +19120,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast753;
-          xfer += iprot->readI32(ecast753);
-          this->state = (LockState::type)ecast753;
+          int32_t ecast761;
+          xfer += iprot->readI32(ecast761);
+          this->state = (LockState::type)ecast761;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18864,9 +19130,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast754;
-          xfer += iprot->readI32(ecast754);
-          this->type = (LockType::type)ecast754;
+          int32_t ecast762;
+          xfer += iprot->readI32(ecast762);
+          this->type = (LockType::type)ecast762;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19082,43 +19348,43 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other755) {
-  lockid = other755.lockid;
-  dbname = other755.dbname;
-  tablename = other755.tablename;
-  partname = other755.partname;
-  state = other755.state;
-  type = other755.type;
-  txnid = other755.txnid;
-  lastheartbeat = other755.lastheartbeat;
-  acquiredat = other755.acquiredat;
-  user = other755.user;
-  hostname = other755.hostname;
-  heartbeatCount = other755.heartbeatCount;
-  agentInfo = other755.agentInfo;
-  blockedByExtId = other755.blockedByExtId;
-  blockedByIntId = other755.blockedByIntId;
-  lockIdInternal = other755.lockIdInternal;
-  __isset = other755.__isset;
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other763) {
+  lockid = other763.lockid;
+  dbname = other763.dbname;
+  tablename = other763.tablename;
+  partname = other763.partname;
+  state = other763.state;
+  type = other763.type;
+  txnid = other763.txnid;
+  lastheartbeat = other763.lastheartbeat;
+  acquiredat = other763.acquiredat;
+  user = other763.user;
+  hostname = other763.hostname;
+  heartbeatCount = other763.heartbeatCount;
+  agentInfo = other763.agentInfo;
+  blockedByExtId = other763.blockedByExtId;
+  blockedByIntId = other763.blockedByIntId;
+  lockIdInternal = other763.lockIdInternal;
+  __isset = other763.__isset;
 }
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other756) {
-  lockid = other756.lockid;
-  dbname = other756.dbname;
-  tablename = other756.tablename;
-  partname = other756.partname;
-  state = other756.state;
-  type = other756.type;
-  txnid = other756.txnid;
-  lastheartbeat = other756.lastheartbeat;
-  acquiredat = other756.acquiredat;
-  user = other756.user;
-  hostname = other756.hostname;
-  heartbeatCount = other756.heartbeatCount;
-  agentInfo = other756.agentInfo;
-  blockedByExtId = other756.blockedByExtId;
-  blockedByIntId = other756.blockedByIntId;
-  lockIdInternal = other756.lockIdInternal;
-  __isset = other756.__isset;
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other764) {
+  lockid = other764.lockid;
+  dbname = other764.dbname;
+  tablename = other764.tablename;
+  partname = other764.partname;
+  state = other764.state;
+  type = other764.type;
+  txnid = other764.txnid;
+  lastheartbeat = other764.lastheartbeat;
+  acquiredat = other764.acquiredat;
+  user = other764.user;
+  hostname = other764.hostname;
+  heartbeatCount = other764.heartbeatCount;
+  agentInfo = other764.agentInfo;
+  blockedByExtId = other764.blockedByExtId;
+  blockedByIntId = other764.blockedByIntId;
+  lockIdInternal = other764.lockIdInternal;
+  __isset = other764.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -19177,14 +19443,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size757;
-            ::apache::thrift::protocol::TType _etype760;
-            xfer += iprot->readListBegin(_etype760, _size757);
-            this->locks.resize(_size757);
-            uint32_t _i761;
-            for (_i761 = 0; _i761 < _size757; ++_i761)
+            uint32_t _size765;
+            ::apache::thrift::protocol::TType _etype768;
+            xfer += iprot->readListBegin(_etype768, _size765);
+            this->locks.resize(_size765);
+            uint32_t _i769;
+            for (_i769 = 0; _i769 < _size765; ++_i769)
             {
-              xfer += this->locks[_i761].read(iprot);
+              xfer += this->locks[_i769].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -19213,10 +19479,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 _iter762;
-    for (_iter762 = this->locks.begin(); _iter762 != this->locks.end(); ++_iter762)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter770;
+    for (_iter770 = this->locks.begin(); _iter770 != this->locks.end(); ++_iter770)
     {
-      xfer += (*_iter762).write(oprot);
+      xfer += (*_iter770).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -19233,13 +19499,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other763) {
-  locks = other763.locks;
-  __isset = other763.__isset;
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other771) {
+  locks = other771.locks;
+  __isset = other771.__isset;
 }
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other764) {
-  locks = other764.locks;
-  __isset = other764.__isset;
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other772) {
+  locks = other772.locks;
+  __isset = other772.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -19340,15 +19606,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other765) {
-  lockid = other765.lockid;
-  txnid = other765.txnid;
-  __isset = other765.__isset;
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other773) {
+  lockid = other773.lockid;
+  txnid = other773.txnid;
+  __isset = other773.__isset;
 }
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other766) {
-  lockid = other766.lockid;
-  txnid = other766.txnid;
-  __isset = other766.__isset;
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other774) {
+  lockid = other774.lockid;
+  txnid = other774.txnid;
+  __isset = other774.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -19451,13 +19717,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other767) {
-  min = other767.min;
-  max = other767.max;
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other775) {
+  min = other775.min;
+  max = other775.max;
 }
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other768) {
-  min = other768.min;
-  max = other768.max;
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other776) {
+  min = other776.min;
+  max = other776.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -19508,15 +19774,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size769;
-            ::apache::thrift::protocol::TType _etype772;
-            xfer += iprot->readSetBegin(_etype772, _size769);
-            uint32_t _i773;
-            for (_i773 = 0; _i773 < _size769; ++_i773)
+            uint32_t _size777;
+            ::apache::thrift::protocol::TType _etype780;
+            xfer += iprot->readSetBegin(_etype780, _size777);
+            uint32_t _i781;
+            for (_i781 = 0; _i781 < _size777; ++_i781)
             {
-              int64_t _elem774;
-              xfer += iprot->readI64(_elem774);
-              this->aborted.insert(_elem774);
+              int64_t _elem782;
+              xfer += iprot->readI64(_elem782);
+              this->aborted.insert(_elem782);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19529,15 +19795,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size775;
-            ::apache::thrift::protocol::TType _etype778;
-            xfer += iprot->readSetBegin(_etype778, _size775);
-            uint32_t _i779;
-            for (_i779 = 0; _i779 < _size775; ++_i779)
+            uint32_t _size783;
+            ::apache::thrift::protocol::TType _etype786;
+            xfer += iprot->readSetBegin(_etype786, _size783);
+            uint32_t _i787;
+            for (_i787 = 0; _i787 < _size783; ++_i787)
             {
-              int64_t _elem780;
-              xfer += iprot->readI64(_elem780);
-              this->nosuch.insert(_elem780);
+              int64_t _elem788;
+              xfer += iprot->readI64(_elem788);
+              this->nosuch.insert(_elem788);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19570,10 +19836,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 _iter781;
-    for (_iter781 = this->aborted.begin(); _iter781 != this->aborted.end(); ++_iter781)
+    std::set<int64_t> ::const_iterator _iter789;
+    for (_iter789 = this->aborted.begin(); _iter789 != this->aborted.end(); ++_iter789)
     {
-      xfer += oprot->writeI64((*_iter781));
+      xfer += oprot->writeI64((*_iter789));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19582,10 +19848,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 _iter782;
-    for (_iter782 = this->nosuch.begin(); _iter782 != this->nosuch.end(); ++_iter782)
+    std::set<int64_t> ::const_iterator _iter790;
+    for (_iter790 = this->nosuch.begin(); _iter790 != this->nosuch.end(); ++_iter790)
     {
-      xfer += oprot->writeI64((*_iter782));
+      xfer += oprot->writeI64((*_iter790));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19602,13 +19868,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other783) {
-  aborted = other783.aborted;
-  nosuch = other783.nosuch;
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other791) {
+  aborted = other791.aborted;
+  nosuch = other791.nosuch;
 }
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other784) {
-  aborted = other784.aborted;
-  nosuch = other784.nosuch;
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other792) {
+  aborted = other792.aborted;
+  nosuch = other792.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -19701,9 +19967,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast785;
-          xfer += iprot->readI32(ecast785);
-          this->type = (CompactionType::type)ecast785;
+          int32_t ecast793;
+          xfer += iprot->readI32(ecast793);
+          this->type = (CompactionType::type)ecast793;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19721,17 +19987,17 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _ktype787;
-            ::apache::thrift::protocol::TType _vtype788;
-            xfer += iprot->readMapBegin(_ktype787, _vtype788, _size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size794;
+            ::apache::thrift::protocol::TType _ktype795;
+            ::apache::thrift::protocol::TType _vtype796;
+            xfer += iprot->readMapBegin(_ktype795, _vtype796, _size794);
+            uint32_t _i798;
+            for (_i798 = 0; _i798 < _size794; ++_i798)
             {
-              std::string _key791;
-              xfer += iprot->readString(_key791);
-              std::string& _val792 = this->properties[_key791];
-              xfer += iprot->readString(_val792);
+              std::string _key799;
+              xfer += iprot->readString(_key799);
+              std::string& _val800 = this->properties[_key799];
+              xfer += iprot->readString(_val800);
             }
             xfer += iprot->readMapEnd();
           }
@@ -19789,11 +20055,11 @@ uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-      std::map<std::string, std::string> ::const_iterator _iter793;
-      for (_iter793 = this->properties.begin(); _iter793 != this->properties.end(); ++_iter793)
+      std::map<std::string, std::string> ::const_iterator _iter801;
+      for (_iter801 = this->properties.begin(); _iter801 != this->properties.end(); ++_iter801)
       {
-        xfer += oprot->writeString(_iter793->first);
-        xfer += oprot->writeString(_iter793->second);
+        xfer += oprot->writeString(_iter801->first);
+        xfer += oprot->writeString(_iter801->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -19815,23 +20081,23 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other794) {
-  dbname = other794.dbname;
-  tablename = other794.tablename;
-  partitionname = other794.partitionname;
-  type = other794.type;
-  runas = other794.runas;
-  properties = other794.properties;
-  __isset = other794.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other795) {
-  dbname = other795.dbname;
-  tablename = other795.tablename;
-  partitionname = other795.partitionname;
-  type = other795.type;
-  runas = other795.runas;
-  properties = other795.properties;
-  __isset = other795.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other802) {
+  dbname = other802.dbname;
+  tablename = other802.tablename;
+  partitionname = other802.partitionname;
+  type = other802.type;
+  runas = other802.runas;
+  properties = other802.properties;
+  __isset = other802.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other803) {
+  dbname = other803.dbname;
+  tablename = other803.tablename;
+  partitionname = other803.partitionname;
+  type = other803.type;
+  runas = other803.runas;
+  properties = other803.properties;
+  __isset = other803.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -19958,15 +20224,15 @@ void swap(CompactionResponse &a, CompactionResponse &b) {
   swap(a.accepted, b.accepted);
 }
 
-CompactionResponse::CompactionResponse(const CompactionResponse& other796) {
-  id = other796.id;
-  state = other796.state;
-  accepted = other796.accepted;
+CompactionResponse::CompactionResponse(const CompactionResponse& other804) {
+  id = other804.id;
+  state = other804.state;
+  accepted = other804.accepted;
 }
-CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other797) {
-  id = other797.id;
-  state = other797.state;
-  accepted = other797.accepted;
+CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other805) {
+  id = other805.id;
+  state = other805.state;
+  accepted = other805.accepted;
   return *this;
 }
 void CompactionResponse::printTo(std::ostream& out) const {
@@ -20027,11 +20293,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other798) {
-  (void) other798;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other806) {
+  (void) other806;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other799) {
-  (void) other799;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other807) {
+  (void) other807;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -20157,9 +20423,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast800;
-          xfer += iprot->readI32(ecast800);
-          this->type = (CompactionType::type)ecast800;
+          int32_t ecast808;
+          xfer += iprot->readI32(ecast808);
+          this->type = (CompactionType::type)ecast808;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20346,37 +20612,37 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other801) {
-  dbname = other801.dbname;
-  tablename = other801.tablename;
-  partitionname = other801.partitionname;
-  type = other801.type;
-  state = other801.state;
-  workerid = other801.workerid;
-  start = other801.start;
-  runAs = other801.runAs;
-  hightestTxnId = other801.hightestTxnId;
-  metaInfo = other801.metaInfo;
-  endTime = other801.endTime;
-  hadoopJobId = other801.hadoopJobId;
-  id = other801.id;
-  __isset = other801.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other802) {
-  dbname = other802.dbname;
-  tablename = other802.tablename;
-  partitionname = other802.partitionname;
-  type = other802.type;
-  state = other802.state;
-  workerid = other802.workerid;
-  start = other802.start;
-  runAs = other802.runAs;
-  hightestTxnId = other802.hightestTxnId;
-  metaInfo = other802.metaInfo;
-  endTime = other802.endTime;
-  hadoopJobId = other802.hadoopJobId;
-  id = other802.id;
-  __isset = other802.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other809) {
+  dbname = other809.dbname;
+  tablename = other809.tablename;
+  partitionname = other809.partitionname;
+  type = other809.type;
+  state = other809.state;
+  workerid = other809.workerid;
+  start = other809.start;
+  runAs = other809.runAs;
+  hightestTxnId = other809.hightestTxnId;
+  metaInfo = other809.metaInfo;
+  endTime = other809.endTime;
+  hadoopJobId = other809.hadoopJobId;
+  id = other809.id;
+  __isset = other809.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other810) {
+  dbname = other810.dbname;
+  tablename = other810.tablename;
+  partitionname = other810.partitionname;
+  type = other810.type;
+  state = other810.state;
+  workerid = other810.workerid;
+  start = other810.start;
+  runAs = other810.runAs;
+  hightestTxnId = other810.hightestTxnId;
+  metaInfo = other810.metaInfo;
+  endTime = other810.endTime;
+  hadoopJobId = other810.hadoopJobId;
+  id = other810.id;
+  __isset = other810.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -20433,14 +20699,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _etype806;
-            xfer += iprot->readListBegin(_etype806, _size803);
-            this->compacts.resize(_size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size811;
+            ::apache::thrift::protocol::TType _etype814;
+            xfer += iprot->readListBegin(_etype814, _size811);
+            this->compacts.resize(_size811);
+            uint32_t _i815;
+            for (_i815 = 0; _i815 < _size811; ++_i815)
             {
-              xfer += this->compacts[_i807].read(iprot);
+              xfer += this->compacts[_i815].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20471,10 +20737,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 _iter808;
-    for (_iter808 = this->compacts.begin(); _iter808 != this->compacts.end(); ++_iter808)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter816;
+    for (_iter816 = this->compacts.begin(); _iter816 != this->compacts.end(); ++_iter816)
     {
-      xfer += (*_iter808).write(oprot);
+      xfer += (*_iter816).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -20490,11 +20756,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other809) {
-  compacts = other809.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other817) {
+  compacts = other817.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other810) {
-  compacts = other810.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other818) {
+  compacts = other818.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -20596,14 +20862,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size811;
-            ::apache::thrift::protocol::TType _etype814;
-            xfer += iprot->readListBegin(_etype814, _size811);
-            this->partitionnames.resize(_size811);
-            uint32_t _i815;
-            for (_i815 = 0; _i815 < _size811; ++_i815)
+            uint32_t _size819;
+            ::apache::thrift::protocol::TType _etype822;
+            xfer += iprot->readListBegin(_etype822, _size819);
+            this->partitionnames.resize(_size819);
+            uint32_t _i823;
+            for (_i823 = 0; _i823 < _size819; ++_i823)
             {
-              xfer += iprot->readString(this->partitionnames[_i815]);
+              xfer += iprot->readString(this->partitionnames[_i823]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20614,9 +20880,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast816;
-          xfer += iprot->readI32(ecast816);
-          this->operationType = (DataOperationType::type)ecast816;
+          int32_t ecast824;
+          xfer += iprot->readI32(ecast824);
+          this->operationType = (DataOperationType::type)ecast824;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20668,10 +20934,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter817;
-    for (_iter817 = this->partitionnames.begin(); _iter817 != this->partitionnames.end(); ++_iter817)
+    std::vector<std::string> ::const_iterator _iter825;
+    for (_iter825 = this->partitionnames.begin(); _iter825 != this->partitionnames.end(); ++_iter825)
     {
-      xfer += oprot->writeString((*_iter817));
+      xfer += oprot->writeString((*_iter825));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20698,23 +20964,23 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other818) {
-  txnid = other818.txnid;
-  writeid = other818.writeid;
-  dbname = other818.dbname;
-  tablename = other818.tablename;
-  partitionnames = other818.partitionnames;
-  operationType = other818.operationType;
-  __isset = other818.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other819) {
-  txnid = other819.txnid;
-  writeid = other819.writeid;
-  dbname = other819.dbname;
-  tablename = other819.tablename;
-  partitionnames = other819.partitionnames;
-  operationType = other819.operationType;
-  __isset = other819.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other826) {
+  txnid = other826.txnid;
+  writeid = other826.writeid;
+  dbname = other826.dbname;
+  tablename = other826.tablename;
+  partitionnames = other826.partitionnames;
+  operationType = other826.operationType;
+  __isset = other826.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other827) {
+  txnid = other827.txnid;
+  writeid = other827.writeid;
+  dbname = other827.dbname;
+  tablename = other827.tablename;
+  partitionnames = other827.partitionnames;
+  operationType = other827.operationType;
+  __isset = other827.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -20897,23 +21163,23 @@ void swap(BasicTxnInfo &a, BasicTxnInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other820) {
-  isnull = other820.isnull;
-  time = other820.time;
-  txnid = other820.txnid;
-  dbname = other820.dbname;
-  tablename = other820.tablename;
-  partitionname = other820.partitionname;
-  __isset = other820.__isset;
-}
-BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other821) {
-  isnull = other821.isnull;
-  time = other821.time;
-  txnid = other821.txnid;
-  dbname = other821.dbname;
-  tablename = other821.tablename;
-  partitionname = other821.partitionname;
-  __isset = other821.__isset;
+BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other828) {
+  isnull = other828.isnull;
+  time = other828.time;
+  txnid = other828.txnid;
+  dbname = other828.dbname;
+  tablename = other828.tablename;
+  partitionname = other828.partitionname;
+  __isset = other828.__isset;
+}
+BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other829) {
+  isnull = other829.isnull;
+  time = other829.time;
+  txnid = other829.txnid;
+  dbname = other829.dbname;
+  tablename = other829.tablename;
+  partitionname = other829.partitionname;
+  __isset = other829.__isset;
   return *this;
 }
 void BasicTxnInfo::printTo(std::ostream& out) const {
@@ -21007,15 +21273,15 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->tablesUsed.clear();
-            uint32_t _size822;
-            ::apache::thrift::protocol::TType _etype825;
-            xfer += iprot->readSetBegin(_etype825, _size822);
-            uint32_t _i826;
-            for (_i826 = 0; _i826 < _size822; ++_i826)
+            uint32_t _size830;
+            ::apache::thrift::protocol::TType _etype833;
+            xfer += iprot->readSetBegin(_etype833, _size830);
+            uint32_t _i834;
+            for (_i834 = 0; _i834 < _size830; ++_i834)
             {
-              std::string _elem827;
-              xfer += iprot->readString(_elem827);
-              this->tablesUsed.insert(_elem827);
+              std::string _elem835;
+              xfer += iprot->readString(_elem835);
+              this->tablesUsed.insert(_elem835);
             }
             xfer += iprot->readSetEnd();
           }
@@ -21072,10 +21338,10 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
-    std::set<std::string> ::const_iterator _iter828;
-    for (_iter828 = this->tablesUsed.begin(); _iter828 != this->tablesUsed.end(); ++_iter828)
+    std::set<std::string> ::const_iterator _iter836;
+    for (_iter836 = this->tablesUsed.begin(); _iter836 != this->tablesUsed.end(); ++_iter836)
     {
-      xfer += oprot->writeString((*_iter828));
+      xfer += oprot->writeString((*_iter836));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -21101,21 +21367,21 @@ void swap(CreationMetadata &a, CreationMetadata &b) {
   swap(a.__isset, b.__isset);
 }
 
-CreationMetadata::CreationMetadata(const CreationMetadata& other829) {
-  catName = other829.catName;
-  dbName = other829.dbName;
-  tblName = other829.tblName;
-  tablesUsed = other829.tablesUsed;
-  validTxnList = other829.validTxnList;
-  __isset = other829.__isset;
-}
-CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other830) {
-  catName = other830.catName;
-  dbName = other830.dbName;
-  tblName = other830.tblName;
-  tablesUsed = other830.tablesUsed;
-  validTxnList = other830.validTxnList;
-  __isset = other830.__isset;
+CreationMetadata::CreationMetadata(const CreationMetadata& other837) {
+  catName = other837.catName;
+  dbName = other837.dbName;
+  tblName = other837.tblName;
+  tablesUsed = other837.tablesUsed;
+  validTxnList = other837.validTxnList;
+  __isset = other837.__isset;
+}
+CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other838) {
+  catName = other838.catName;
+  dbName = other838.dbName;
+  tblName = other838.tblName;
+  tablesUsed = other838.tablesUsed;
+  validTxnList = other838.validTxnList;
+  __isset = other838.__isset;
   return *this;
 }
 void CreationMetadata::printTo(std::ostream& out) const {
@@ -21221,15 +21487,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other831) {
-  lastEvent = other831.lastEvent;
-  maxEvents = other831.maxEvents;
-  __isset = other831.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other839) {
+  lastEvent = other839.lastEvent;
+  maxEvents = other839.maxEvents;
+  __isset = other839.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other832) {
-  lastEvent = other832.lastEvent;
-  maxEvents = other832.maxEvents;
-  __isset = other832.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other840) {
+  lastEvent = other840.lastEvent;
+  maxEvents = other840.maxEvents;
+  __isset = other840.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -21449,27 +21715,27 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other833) {
-  eventId = other833.eventId;
-  eventTime = other833.eventTime;
-  eventType = other833.eventType;
-  dbName = other833.dbName;
-  tableName = other833.tableName;
-  message = other833.message;
-  messageFormat = other833.messageFormat;
-  catName = other833.catName;
-  __isset = other833.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other834) {
-  eventId = other834.eventId;
-  eventTime = other834.eventTime;
-  eventType = other834.eventType;
-  dbName = other834.dbName;
-  tableName = other834.tableName;
-  message = other834.message;
-  messageFormat = other834.messageFormat;
-  catName = other834.catName;
-  __isset = other834.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other841) {
+  eventId = other841.eventId;
+  eventTime = other841.eventTime;
+  eventType = other841.eventType;
+  dbName = other841.dbName;
+  tableName = other841.tableName;
+  message = other841.message;
+  messageFormat = other841.messageFormat;
+  catName = other841.catName;
+  __isset = other841.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other842) {
+  eventId = other842.eventId;
+  eventTime = other842.eventTime;
+  eventType = other842.eventType;
+  dbName = other842.dbName;
+  tableName = other842.tableName;
+  message = other842.message;
+  messageFormat = other842.messageFormat;
+  catName = other842.catName;
+  __isset = other842.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -21521,14 +21787,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size835;
-            ::apache::thrift::protocol::TType _etype838;
-            xfer += iprot->readListBegin(_etype838, _size835);
-            this->events.resize(_size835);
-            uint32_t _i839;
-            for (_i839 = 0; _i839 < _size835; ++_i839)
+            uint32_t _size843;
+            ::apache::thrift::protocol::TType _etype846;
+            xfer += iprot->readListBegin(_etype846, _size843);
+            this->events.resize(_size843);
+            uint32_t _i847;
+            for (_i847 = 0; _i847 < _size843; ++_i847)
             {
-              xfer += this->events[_i839].read(iprot);
+              xfer += this->events[_i847].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21559,10 +21825,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 _iter840;
-    for (_iter840 = this->events.begin(); _iter840 != this->events.end(); ++_iter840)
+    std::vector<NotificationEvent> ::const_iterator _iter848;
+    for (_iter848 = this->events.begin(); _iter848 != this->events.end(); ++_iter848)
     {
-      xfer += (*_iter840).write(oprot);
+      xfer += (*_iter848).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -21578,11 +21844,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other841) {
-  events = other841.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other849) {
+  events = other849.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other842) {
-  events = other842.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other850) {
+  events = other850.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -21664,11 +21930,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other843) {
-  eventId = other843.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other851) {
+  eventId = other851.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other844) {
-  eventId = other844.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other852) {
+  eventId = other852.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -21790,17 +22056,17 @@ void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other845) {
-  fromEventId = other845.fromEventId;
-  dbName = other845.dbName;
-  catName = other845.catName;
-  __isset = other845.__isset;
+NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other853) {
+  fromEventId = other853.fromEventId;
+  dbName = other853.dbName;
+  catName = other853.catName;
+  __isset = other853.__isset;
 }
-NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other846) {
-  fromEventId = other846.fromEventId;
-  dbName = other846.dbName;
-  catName = other846.catName;
-  __isset = other846.__isset;
+NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other854) {
+  fromEventId = other854.fromEventId;
+  dbName = other854.dbName;
+  catName = other854.catName;
+  __isset = other854.__isset;
   return *this;
 }
 void NotificationEventsCountRequest::printTo(std::ostream& out) const {
@@ -21884,11 +22150,11 @@ void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b
   swap(a.eventsCount, b.eventsCount);
 }
 
-NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other847) {
-  eventsCount = other847.eventsCount;
+NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other855) {
+  eventsCount = other855.eventsCount;
 }
-NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other848) {
-  eventsCount = other848.eventsCount;
+NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other856) {
+  eventsCount = other856.eventsCount;
   return *this;
 }
 void NotificationEventsCountResponse::printTo(std::ostream& out) const {
@@ -21917,6 +22183,11 @@ void InsertEventRequestData::__set_filesAddedChecksum(const std::vector<std::str
 __isset.filesAddedChecksum = true;
 }
 
+void InsertEventRequestData::__set_subDirectoryList(const std::vector<std::string> & val) {
+  this->subDirectoryList = val;
+__isset.subDirectoryList = true;
+}
+
 uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -21951,14 +22222,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size849;
-            ::apache::thrift::protocol::TType _etype852;
-            xfer += iprot->readListBegin(_etype852, _size849);
-            this->filesAdded.resize(_size849);
-            uint32_t _i853;
-            for (_i853 = 0; _i853 < _size849; ++_i853)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            this->filesAdded.resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += iprot->readString(this->filesAdded[_i853]);
+              xfer += iprot->readString(this->filesAdded[_i861]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21971,14 +22242,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAddedChecksum.clear();
-            uint32_t _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            this->filesAddedChecksum.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->filesAddedChecksum.resize(_size862);
+            uint32_t _i866;
+            for (_i866 = 0; _i866 < _size862; ++_i866)
             {
-              xfer += iprot->readString(this->filesAddedChecksum[_i858]);
+              xfer += iprot->readString(this->filesAddedChecksum[_i866]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21987,6 +22258,26 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->subDirectoryList.clear();
+            uint32_t _size867;
+            ::apache::thrift::protocol::TType _etype870;
+            xfer += iprot->readListBegin(_etype870, _size867);
+            this->subDirectoryList.resize(_size867);
+            uint32_t _i871;
+            for (_i871 = 0; _i871 < _size867; ++_i871)
+            {
+              xfer += iprot->readString(this->subDirectoryList[_i871]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.subDirectoryList = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -22014,10 +22305,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter859;
-    for (_iter859 = this->filesAdded.begin(); _iter859 != this->filesAdded.end(); ++_iter859)
+    std::vector<std::string> ::const_iterator _iter872;
+    for (_iter872 = this->filesAdded.begin(); _iter872 != this->filesAdded.end(); ++_iter872)
     {
-      xfer += oprot->writeString((*_iter859));
+      xfer += oprot->writeString((*_iter872));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22027,10 +22318,23 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
     xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAddedChecksum.size()));
-      std::vector<std::string> ::const_iterator _iter860;
-      for (_iter860 = this->filesAddedChecksum.begin(); _iter860 != this->filesAddedChecksum.end(); ++_iter860)
+      std::vector<std::string> ::const_iterator _iter873;
+      for (_iter873 = this->filesAddedChecksum.begin(); _iter873 != this->filesAddedChecksum.end(); ++_iter873)
+      {
+        xfer += oprot->writeString((*_iter873));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.subDirectoryList) {
+    xfer += oprot->writeFieldBegin("subDirectoryList", ::apache::thrift::protocol::T_LIST, 4);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->subDirectoryList.size()));
+      std::vector<std::string> ::const_iterator _iter874;
+      for (_iter874 = this->subDirectoryList.begin(); _iter874 != this->subDirectoryList.end(); ++_iter874)
       {
-        xfer += oprot->writeString((*_iter860));
+        xfer += oprot->writeString((*_iter874));
       }
       xfer += oprot->writeListEnd();
     }
@@ -22046,20 +22350,23 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.replace, b.replace);
   swap(a.filesAdded, b.filesAdded);
   swap(a.filesAddedChecksum, b.filesAddedChecksum);
+  swap(a.subDirectoryList, b.subDirectoryList);
   swap(a.__isset, b.__isset);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other861) {
-  replace = other861.replace;
-  filesAdded = other861.filesAdded;
-  filesAddedChecksum = other861.filesAddedChecksum;
-  __isset = other861.__isset;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other875) {
+  replace = other875.replace;
+  filesAdded = other875.filesAdded;
+  filesAddedChecksum = other875.filesAddedChecksum;
+  subDirectoryList = other875.subDirectoryList;
+  __isset = other875.__isset;
 }
-InsertEventRequest

<TRUNCATED>