You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/06/28 22:48:47 UTC

[51/52] [abbrv] hive git commit: HIVE-19867 : Test and verify Concurrent INSERT (Steve Yeom)

HIVE-19867 : Test and verify Concurrent INSERT (Steve Yeom)


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

Branch: refs/heads/master-txnstats
Commit: 222b4aca081529d534f94cece1f9af79a0fb3dda
Parents: b51b47c
Author: sergey <se...@apache.org>
Authored: Thu Jun 28 15:23:08 2018 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Jun 28 15:23:08 2018 -0700

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |   4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  42 ++++--
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  11 ++
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |   5 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |  14 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  33 +++--
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 132 +++++++++++++++--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  60 ++++++--
 .../metastore/api/AlterPartitionsRequest.java   | 117 +++++++++++++--
 .../hadoop/hive/metastore/api/Partition.java    | 129 +++++++++++++++--
 .../hive/metastore/api/PartitionSpec.java       | 129 +++++++++++++++--
 .../api/SetPartitionsStatsRequest.java          | 117 +++++++++++++--
 .../apache/hadoop/hive/metastore/api/Table.java | 129 +++++++++++++++--
 .../src/gen/thrift/gen-php/metastore/Types.php  | 143 +++++++++++++++++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 113 +++++++++++----
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  26 ++--
 .../hadoop/hive/metastore/AlterHandler.java     |   3 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java |   9 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   9 +-
 .../hive/metastore/HiveMetaStoreClient.java     |  11 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   6 +-
 .../hadoop/hive/metastore/ObjectStore.java      |  53 +++++--
 .../apache/hadoop/hive/metastore/RawStore.java  |   3 +-
 .../hive/metastore/cache/CachedStore.java       |   4 +-
 .../hadoop/hive/metastore/model/MPartition.java |   9 ++
 .../hadoop/hive/metastore/model/MTable.java     |   9 ++
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   4 +-
 .../src/main/resources/package.jdo              |   6 +
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |   4 +-
 .../sql/derby/upgrade-3.1.0-to-4.0.0.derby.sql  |   2 +
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |   1 +
 .../sql/mssql/upgrade-3.1.0-to-4.0.0.mssql.sql  |   2 +
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |   2 +
 .../sql/mysql/upgrade-3.1.0-to-4.0.0.mysql.sql  |   2 +
 .../sql/oracle/hive-schema-4.0.0.oracle.sql     |   2 +
 .../oracle/upgrade-3.1.0-to-4.0.0.oracle.sql    |   2 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |   1 +
 .../upgrade-3.1.0-to-4.0.0.postgres.sql         |   2 +
 .../src/main/thrift/hive_metastore.thrift       |  21 +--
 .../DummyRawStoreControlledCommit.java          |   4 +-
 .../DummyRawStoreForJdoConnection.java          |   2 +-
 .../HiveMetaStoreClientPreCatalog.java          |   4 +-
 .../metastore/client/TestAlterPartitions.java   |   2 +-
 .../org/apache/hive/common/util/TxnIdUtils.java |  23 ++-
 44 files changed, 1198 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 580bae9..31d9b71 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
@@ -396,9 +396,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public void alterPartitions(String catName, String dbName, String tblName,
                               List<List<String>> partValsList, List<Partition> newParts,
-                              long txnId, String writeIdList)
+                              long txnId, String writeIdList, long writeId)
       throws InvalidObjectException, MetaException {
-    objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts, txnId, writeIdList);
+    objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts, txnId, writeIdList, writeId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 07ed709..f126550 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
@@ -1625,13 +1625,15 @@ public class AcidUtils {
 
   public static class TableSnapshot {
     private long txnId;
+    private long writeId;
     private String validWriteIdList;
 
     public TableSnapshot() {
     }
 
-    public TableSnapshot(long txnId, String validWriteIdList) {
+    public TableSnapshot(long txnId, long writeId, String validWriteIdList) {
       this.txnId = txnId;
+      this.writeId = writeId;
       this.validWriteIdList = validWriteIdList;
     }
 
@@ -1647,6 +1649,14 @@ public class AcidUtils {
       this.txnId = txnId;
     }
 
+    public long getWriteId() {
+      return writeId;
+    }
+
+    public void setWriteId(long writeId) {
+      this.writeId = writeId;
+    }
+
     public void setValidWriteIdList(String validWriteIdList) {
       this.validWriteIdList = validWriteIdList;
     }
@@ -1657,21 +1667,20 @@ public class AcidUtils {
     }
   }
 
-  /**
-   * Create a TableShopshot with the given "conf"
-   * for the table of the given "tbl".
-   *
-   * @param conf
-   * @param tbl
-   * @return TableSnapshot on success, null on failure
-   * @throws LockException
-   */
   public static TableSnapshot getTableSnapshot(
-      Configuration conf, Table tbl) throws LockException {
+          Configuration conf,
+          Table tbl) throws LockException {
+    return getTableSnapshot(conf, tbl, false);
+  }
+
+
+  public static TableSnapshot getTableSnapshot(
+      Configuration conf, Table tbl, boolean isStatsUpdater) throws LockException {
     if (!isTransactionalTable(tbl)) {
       return null;
     } else {
-      long txnId = 0;
+      long txnId = -1;
+      long writeId = -1;
       ValidWriteIdList validWriteIdList = null;
 
       HiveTxnManager sessionTxnMgr = SessionState.get().getTxnMgr();
@@ -1682,6 +1691,13 @@ public class AcidUtils {
       String fullTableName = getFullTableName(tbl.getDbName(), tbl.getTableName());
       if (txnId > 0 && isTransactionalTable(tbl)) {
         validWriteIdList = getTableValidWriteIdList(conf, fullTableName);
+        if (isStatsUpdater) {
+          // TODO# it should be invalid to update stats without write ID...
+          //       Why would there be a stats updater that doesn't have a write ID?
+          writeId = SessionState.get().getTxnMgr() != null ?
+                  SessionState.get().getTxnMgr().getAllocatedTableWriteId(
+                    tbl.getDbName(), tbl.getTableName()) : -1;
+        }
 
 
         if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST)
@@ -1696,7 +1712,7 @@ public class AcidUtils {
           throw new AssertionError("Cannot find valid write ID list for " + tbl.getTableName());
         }
       }
-      return new TableSnapshot(txnId,
+      return new TableSnapshot(txnId, writeId,
           validWriteIdList != null ? validWriteIdList.toString() : null);
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 9104786..18eed52 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.LockTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
+import org.apache.hadoop.hive.ql.session.*;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.apache.thrift.TException;
@@ -1020,6 +1021,16 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     return getTableWriteId(dbName, tableName, true);
   }
 
+  @Override
+  public long getAllocatedTableWriteId(String dbName, String tableName) throws LockException {
+    assert isTxnOpen();
+    // Calls getTableWriteId() with allocateIfNotYet being false
+    // to return 0 if the dbName:tableName's writeId is yet allocated.
+    // This happens when the current context is before
+    // Driver.acquireLocks() is called.
+    return getTableWriteId(dbName, tableName, false);
+  }
+
   private long getTableWriteId(
       String dbName, String tableName, boolean allocateIfNotYet) throws LockException {
     String fullTableName = AcidUtils.getFullTableName(dbName, tableName);

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 78bb303..2398419 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
@@ -79,6 +79,11 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
+  public long getAllocatedTableWriteId(String dbName, String tableName) throws LockException {
+    return 0L;
+  }
+
+  @Override
   public void replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
                                              List<TxnToWriteId> srcTxnToWriteIdList) throws LockException {
     return;

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 9ea40f4..fcfa58b 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
@@ -276,7 +276,19 @@ public interface HiveTxnManager {
    * if {@code isTxnOpen()}, returns the table write ID associated with current active transaction.
    */
   long getTableWriteId(String dbName, String tableName) throws LockException;
-  /**
+
+ /**
+  * if {@code isTxnOpen()}, returns the already allocated table write ID of the table with
+  * the given "dbName.tableName" for the current active transaction.
+  * If not allocated, then returns 0.
+  * @param dbName
+  * @param tableName
+  * @return 0 if not yet allocated
+  * @throws LockException
+  */
+ public long getAllocatedTableWriteId(String dbName, String tableName) throws LockException;
+
+ /**
    * Allocates write id for each transaction in the list.
    * @param dbName database name
    * @param tableName the name of the table to allocate the write id

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 37bcc03..9511d63 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
@@ -641,7 +641,7 @@ public class Hive {
 
       // Take a table snapshot and set it to newTbl.
       if (transactional) {
-        setTableSnapshotForTransactionalTable(conf, newTbl);
+        setTableSnapshotForTransactionalTable(conf, newTbl, true);
       }
 
       getMSC().alter_table_with_environmentContext(dbName, tblName, newTbl.getTTable(), environmentContext);
@@ -724,7 +724,7 @@ public class Hive {
         newPart.setLocation(location);
       }
       if (transactional) {
-        setTableSnapshotForTransactionalPartition(conf, newPart);
+        setTableSnapshotForTransactionalPartition(conf, newPart, true);
       }
       getSynchronizedMSC().alter_partition(dbName, tblName, newPart.getTPartition(), environmentContext);
 
@@ -769,7 +769,7 @@ public class Hive {
     try {
       AcidUtils.TableSnapshot tableSnapshot = null;
       if (transactional) {
-        tableSnapshot = AcidUtils.getTableSnapshot(conf, newParts.get(0).getTable());
+        tableSnapshot = AcidUtils.getTableSnapshot(conf, newParts.get(0).getTable(), true);
       }
       // Remove the DDL time so that it gets refreshed
       for (Partition tmpPart: newParts) {
@@ -785,7 +785,8 @@ public class Hive {
       }
       getMSC().alter_partitions(names[0], names[1], newTParts, environmentContext,
           tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
-          tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
+          tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null,
+          tableSnapshot != null ? tableSnapshot.getWriteId() : -1    );
     } catch (MetaException e) {
       throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     } catch (TException e) {
@@ -917,7 +918,7 @@ public class Hive {
         }
       }
       // Set table snapshot to api.Table to make it persistent.
-      setTableSnapshotForTransactionalTable(conf, tbl);
+      setTableSnapshotForTransactionalTable(conf, tbl, true);
       if (primaryKeys == null && foreignKeys == null
               && uniqueConstraints == null && notNullConstraints == null && defaultConstraints == null
           && checkConstraints == null) {
@@ -1813,7 +1814,7 @@ public class Hive {
       Partition newTPart = oldPart != null ? oldPart : new Partition(tbl, partSpec, newPartPath);
       alterPartitionSpecInMemory(tbl, partSpec, newTPart.getTPartition(), inheritTableSpecs, newPartPath.toString());
       validatePartition(newTPart);
-      setTableSnapshotForTransactionalPartition(conf, newTPart);
+      setTableSnapshotForTransactionalPartition(conf, newTPart, true);
 
       // 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
@@ -4448,9 +4449,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
       ColumnStatisticsDesc statsDesc = colStat.getStatsDesc();
       Table tbl = getTable(statsDesc.getDbName(), statsDesc.getTableName());
 
-      AcidUtils.TableSnapshot tableSnapshot  = AcidUtils.getTableSnapshot(conf, tbl);
+      AcidUtils.TableSnapshot tableSnapshot  = AcidUtils.getTableSnapshot(conf, tbl, true);
       request.setTxnId(tableSnapshot != null ? tableSnapshot.getTxnId() : 0);
       request.setValidWriteIdList(tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
+      request.setWriteId(tableSnapshot != null ? tableSnapshot.getWriteId() : 0);
       return getMSC().setPartitionColumnStatistics(request);
     } catch (Exception e) {
       LOG.debug(StringUtils.stringifyException(e));
@@ -5331,24 +5333,33 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   private void setTableSnapshotForTransactionalTable(
-      HiveConf conf, Table newTbl)
+      HiveConf conf, Table newTbl, boolean isStatsUpdater)
       throws LockException {
 
     org.apache.hadoop.hive.metastore.api.Table newTTbl = newTbl.getTTable();
-    AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, newTbl);
+    AcidUtils.TableSnapshot tableSnapshot =
+            AcidUtils.getTableSnapshot(conf, newTbl, isStatsUpdater);
 
     newTTbl.setTxnId(tableSnapshot != null ? tableSnapshot.getTxnId() : -1);
     newTTbl.setValidWriteIdList(
         tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
+    if (isStatsUpdater) {
+      newTTbl.setWriteId(tableSnapshot != null ? tableSnapshot.getWriteId() : -1);
+    }
   }
 
-  private void setTableSnapshotForTransactionalPartition(HiveConf conf, Partition partition)
+  private void setTableSnapshotForTransactionalPartition(
+      HiveConf conf, Partition partition, boolean isStatsUpdater)
       throws LockException {
 
-    AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, partition.getTable());
+    AcidUtils.TableSnapshot tableSnapshot =
+            AcidUtils.getTableSnapshot(conf, partition.getTable(), isStatsUpdater);
     org.apache.hadoop.hive.metastore.api.Partition tpartition = partition.getTPartition();
     tpartition.setTxnId(tableSnapshot != null ? tableSnapshot.getTxnId() : -1);
     tpartition.setValidWriteIdList(
         tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
+    if (isStatsUpdater) {
+      tpartition.setWriteId(tableSnapshot != null ? tableSnapshot.getWriteId() : -1);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 c2d6a56..da14ac1 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
@@ -6452,6 +6452,11 @@ void Table::__set_txnId(const int64_t val) {
 __isset.txnId = true;
 }
 
+void Table::__set_writeId(const int64_t val) {
+  this->writeId = val;
+__isset.writeId = true;
+}
+
 void Table::__set_validWriteIdList(const std::string& val) {
   this->validWriteIdList = val;
 __isset.validWriteIdList = true;
@@ -6665,6 +6670,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 20:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          this->__isset.writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 21:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->validWriteIdList);
           this->__isset.validWriteIdList = true;
@@ -6672,7 +6685,7 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 21:
+      case 22:
         if (ftype == ::apache::thrift::protocol::T_I32) {
           int32_t ecast249;
           xfer += iprot->readI32(ecast249);
@@ -6799,13 +6812,18 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeI64(this->txnId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeId) {
+    xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 20);
+    xfer += oprot->writeI64(this->writeId);
+    xfer += oprot->writeFieldEnd();
+  }
   if (this->__isset.validWriteIdList) {
-    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 20);
+    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 21);
     xfer += oprot->writeString(this->validWriteIdList);
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.isStatsCompliant) {
-    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 21);
+    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 22);
     xfer += oprot->writeI32((int32_t)this->isStatsCompliant);
     xfer += oprot->writeFieldEnd();
   }
@@ -6835,6 +6853,7 @@ void swap(Table &a, Table &b) {
   swap(a.catName, b.catName);
   swap(a.ownerType, b.ownerType);
   swap(a.txnId, b.txnId);
+  swap(a.writeId, b.writeId);
   swap(a.validWriteIdList, b.validWriteIdList);
   swap(a.isStatsCompliant, b.isStatsCompliant);
   swap(a.__isset, b.__isset);
@@ -6860,6 +6879,7 @@ Table::Table(const Table& other252) {
   catName = other252.catName;
   ownerType = other252.ownerType;
   txnId = other252.txnId;
+  writeId = other252.writeId;
   validWriteIdList = other252.validWriteIdList;
   isStatsCompliant = other252.isStatsCompliant;
   __isset = other252.__isset;
@@ -6884,6 +6904,7 @@ Table& Table::operator=(const Table& other253) {
   catName = other253.catName;
   ownerType = other253.ownerType;
   txnId = other253.txnId;
+  writeId = other253.writeId;
   validWriteIdList = other253.validWriteIdList;
   isStatsCompliant = other253.isStatsCompliant;
   __isset = other253.__isset;
@@ -6911,6 +6932,7 @@ void Table::printTo(std::ostream& out) const {
   out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ", " << "ownerType="; (__isset.ownerType ? (out << to_string(ownerType)) : (out << "<null>"));
   out << ", " << "txnId="; (__isset.txnId ? (out << to_string(txnId)) : (out << "<null>"));
+  out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "<null>"));
   out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "<null>"));
   out << ", " << "isStatsCompliant="; (__isset.isStatsCompliant ? (out << to_string(isStatsCompliant)) : (out << "<null>"));
   out << ")";
@@ -6964,6 +6986,11 @@ void Partition::__set_txnId(const int64_t val) {
 __isset.txnId = true;
 }
 
+void Partition::__set_writeId(const int64_t val) {
+  this->writeId = val;
+__isset.writeId = true;
+}
+
 void Partition::__set_validWriteIdList(const std::string& val) {
   this->validWriteIdList = val;
 __isset.validWriteIdList = true;
@@ -7103,6 +7130,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 11:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          this->__isset.writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 12:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->validWriteIdList);
           this->__isset.validWriteIdList = true;
@@ -7110,7 +7145,7 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 12:
+      case 13:
         if (ftype == ::apache::thrift::protocol::T_I32) {
           int32_t ecast266;
           xfer += iprot->readI32(ecast266);
@@ -7197,13 +7232,18 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeI64(this->txnId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeId) {
+    xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 11);
+    xfer += oprot->writeI64(this->writeId);
+    xfer += oprot->writeFieldEnd();
+  }
   if (this->__isset.validWriteIdList) {
-    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 11);
+    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 12);
     xfer += oprot->writeString(this->validWriteIdList);
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.isStatsCompliant) {
-    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 12);
+    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 13);
     xfer += oprot->writeI32((int32_t)this->isStatsCompliant);
     xfer += oprot->writeFieldEnd();
   }
@@ -7224,6 +7264,7 @@ void swap(Partition &a, Partition &b) {
   swap(a.privileges, b.privileges);
   swap(a.catName, b.catName);
   swap(a.txnId, b.txnId);
+  swap(a.writeId, b.writeId);
   swap(a.validWriteIdList, b.validWriteIdList);
   swap(a.isStatsCompliant, b.isStatsCompliant);
   swap(a.__isset, b.__isset);
@@ -7240,6 +7281,7 @@ Partition::Partition(const Partition& other269) {
   privileges = other269.privileges;
   catName = other269.catName;
   txnId = other269.txnId;
+  writeId = other269.writeId;
   validWriteIdList = other269.validWriteIdList;
   isStatsCompliant = other269.isStatsCompliant;
   __isset = other269.__isset;
@@ -7255,6 +7297,7 @@ Partition& Partition::operator=(const Partition& other270) {
   privileges = other270.privileges;
   catName = other270.catName;
   txnId = other270.txnId;
+  writeId = other270.writeId;
   validWriteIdList = other270.validWriteIdList;
   isStatsCompliant = other270.isStatsCompliant;
   __isset = other270.__isset;
@@ -7273,6 +7316,7 @@ void Partition::printTo(std::ostream& out) const {
   out << ", " << "privileges="; (__isset.privileges ? (out << to_string(privileges)) : (out << "<null>"));
   out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ", " << "txnId="; (__isset.txnId ? (out << to_string(txnId)) : (out << "<null>"));
+  out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "<null>"));
   out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "<null>"));
   out << ", " << "isStatsCompliant="; (__isset.isStatsCompliant ? (out << to_string(isStatsCompliant)) : (out << "<null>"));
   out << ")";
@@ -7779,6 +7823,11 @@ void PartitionSpec::__set_txnId(const int64_t val) {
 __isset.txnId = true;
 }
 
+void PartitionSpec::__set_writeId(const int64_t val) {
+  this->writeId = val;
+__isset.writeId = true;
+}
+
 void PartitionSpec::__set_validWriteIdList(const std::string& val) {
   this->validWriteIdList = val;
 __isset.validWriteIdList = true;
@@ -7867,6 +7916,14 @@ uint32_t PartitionSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 8:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          this->__isset.writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 9:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->validWriteIdList);
           this->__isset.validWriteIdList = true;
@@ -7874,7 +7931,7 @@ uint32_t PartitionSpec::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 9:
+      case 10:
         if (ftype == ::apache::thrift::protocol::T_I32) {
           int32_t ecast303;
           xfer += iprot->readI32(ecast303);
@@ -7933,13 +7990,18 @@ uint32_t PartitionSpec::write(::apache::thrift::protocol::TProtocol* oprot) cons
     xfer += oprot->writeI64(this->txnId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeId) {
+    xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 8);
+    xfer += oprot->writeI64(this->writeId);
+    xfer += oprot->writeFieldEnd();
+  }
   if (this->__isset.validWriteIdList) {
-    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 8);
+    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 9);
     xfer += oprot->writeString(this->validWriteIdList);
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.isStatsCompliant) {
-    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 9);
+    xfer += oprot->writeFieldBegin("isStatsCompliant", ::apache::thrift::protocol::T_I32, 10);
     xfer += oprot->writeI32((int32_t)this->isStatsCompliant);
     xfer += oprot->writeFieldEnd();
   }
@@ -7957,6 +8019,7 @@ void swap(PartitionSpec &a, PartitionSpec &b) {
   swap(a.partitionList, b.partitionList);
   swap(a.catName, b.catName);
   swap(a.txnId, b.txnId);
+  swap(a.writeId, b.writeId);
   swap(a.validWriteIdList, b.validWriteIdList);
   swap(a.isStatsCompliant, b.isStatsCompliant);
   swap(a.__isset, b.__isset);
@@ -7970,6 +8033,7 @@ PartitionSpec::PartitionSpec(const PartitionSpec& other304) {
   partitionList = other304.partitionList;
   catName = other304.catName;
   txnId = other304.txnId;
+  writeId = other304.writeId;
   validWriteIdList = other304.validWriteIdList;
   isStatsCompliant = other304.isStatsCompliant;
   __isset = other304.__isset;
@@ -7982,6 +8046,7 @@ PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other305) {
   partitionList = other305.partitionList;
   catName = other305.catName;
   txnId = other305.txnId;
+  writeId = other305.writeId;
   validWriteIdList = other305.validWriteIdList;
   isStatsCompliant = other305.isStatsCompliant;
   __isset = other305.__isset;
@@ -7997,6 +8062,7 @@ void PartitionSpec::printTo(std::ostream& out) const {
   out << ", " << "partitionList="; (__isset.partitionList ? (out << to_string(partitionList)) : (out << "<null>"));
   out << ", " << "catName="; (__isset.catName ? (out << to_string(catName)) : (out << "<null>"));
   out << ", " << "txnId="; (__isset.txnId ? (out << to_string(txnId)) : (out << "<null>"));
+  out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "<null>"));
   out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "<null>"));
   out << ", " << "isStatsCompliant="; (__isset.isStatsCompliant ? (out << to_string(isStatsCompliant)) : (out << "<null>"));
   out << ")";
@@ -10317,6 +10383,11 @@ void SetPartitionsStatsRequest::__set_txnId(const int64_t val) {
 __isset.txnId = true;
 }
 
+void SetPartitionsStatsRequest::__set_writeId(const int64_t val) {
+  this->writeId = val;
+__isset.writeId = true;
+}
+
 void SetPartitionsStatsRequest::__set_validWriteIdList(const std::string& val) {
   this->validWriteIdList = val;
 __isset.validWriteIdList = true;
@@ -10381,6 +10452,14 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol*
         }
         break;
       case 4:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          this->__isset.writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->validWriteIdList);
           this->__isset.validWriteIdList = true;
@@ -10429,8 +10508,13 @@ uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol*
     xfer += oprot->writeI64(this->txnId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeId) {
+    xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 4);
+    xfer += oprot->writeI64(this->writeId);
+    xfer += oprot->writeFieldEnd();
+  }
   if (this->__isset.validWriteIdList) {
-    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 4);
+    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 5);
     xfer += oprot->writeString(this->validWriteIdList);
     xfer += oprot->writeFieldEnd();
   }
@@ -10444,6 +10528,7 @@ void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) {
   swap(a.colStats, b.colStats);
   swap(a.needMerge, b.needMerge);
   swap(a.txnId, b.txnId);
+  swap(a.writeId, b.writeId);
   swap(a.validWriteIdList, b.validWriteIdList);
   swap(a.__isset, b.__isset);
 }
@@ -10452,6 +10537,7 @@ SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsReq
   colStats = other354.colStats;
   needMerge = other354.needMerge;
   txnId = other354.txnId;
+  writeId = other354.writeId;
   validWriteIdList = other354.validWriteIdList;
   __isset = other354.__isset;
 }
@@ -10459,6 +10545,7 @@ SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartiti
   colStats = other355.colStats;
   needMerge = other355.needMerge;
   txnId = other355.txnId;
+  writeId = other355.writeId;
   validWriteIdList = other355.validWriteIdList;
   __isset = other355.__isset;
   return *this;
@@ -10469,6 +10556,7 @@ void SetPartitionsStatsRequest::printTo(std::ostream& out) const {
   out << "colStats=" << to_string(colStats);
   out << ", " << "needMerge="; (__isset.needMerge ? (out << to_string(needMerge)) : (out << "<null>"));
   out << ", " << "txnId="; (__isset.txnId ? (out << to_string(txnId)) : (out << "<null>"));
+  out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "<null>"));
   out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "<null>"));
   out << ")";
 }
@@ -31831,6 +31919,11 @@ void AlterPartitionsRequest::__set_txnId(const int64_t val) {
 __isset.txnId = true;
 }
 
+void AlterPartitionsRequest::__set_writeId(const int64_t val) {
+  this->writeId = val;
+__isset.writeId = true;
+}
+
 void AlterPartitionsRequest::__set_validWriteIdList(const std::string& val) {
   this->validWriteIdList = val;
 __isset.validWriteIdList = true;
@@ -31914,6 +32007,14 @@ uint32_t AlterPartitionsRequest::read(::apache::thrift::protocol::TProtocol* ipr
         }
         break;
       case 6:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          this->__isset.writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
         if (ftype == ::apache::thrift::protocol::T_STRING) {
           xfer += iprot->readString(this->validWriteIdList);
           this->__isset.validWriteIdList = true;
@@ -31975,8 +32076,13 @@ uint32_t AlterPartitionsRequest::write(::apache::thrift::protocol::TProtocol* op
     xfer += oprot->writeI64(this->txnId);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeId) {
+    xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 6);
+    xfer += oprot->writeI64(this->writeId);
+    xfer += oprot->writeFieldEnd();
+  }
   if (this->__isset.validWriteIdList) {
-    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeFieldBegin("validWriteIdList", ::apache::thrift::protocol::T_STRING, 7);
     xfer += oprot->writeString(this->validWriteIdList);
     xfer += oprot->writeFieldEnd();
   }
@@ -31992,6 +32098,7 @@ void swap(AlterPartitionsRequest &a, AlterPartitionsRequest &b) {
   swap(a.partitions, b.partitions);
   swap(a.environmentContext, b.environmentContext);
   swap(a.txnId, b.txnId);
+  swap(a.writeId, b.writeId);
   swap(a.validWriteIdList, b.validWriteIdList);
   swap(a.__isset, b.__isset);
 }
@@ -32002,6 +32109,7 @@ AlterPartitionsRequest::AlterPartitionsRequest(const AlterPartitionsRequest& oth
   partitions = other1187.partitions;
   environmentContext = other1187.environmentContext;
   txnId = other1187.txnId;
+  writeId = other1187.writeId;
   validWriteIdList = other1187.validWriteIdList;
   __isset = other1187.__isset;
 }
@@ -32011,6 +32119,7 @@ AlterPartitionsRequest& AlterPartitionsRequest::operator=(const AlterPartitionsR
   partitions = other1188.partitions;
   environmentContext = other1188.environmentContext;
   txnId = other1188.txnId;
+  writeId = other1188.writeId;
   validWriteIdList = other1188.validWriteIdList;
   __isset = other1188.__isset;
   return *this;
@@ -32023,6 +32132,7 @@ void AlterPartitionsRequest::printTo(std::ostream& out) const {
   out << ", " << "partitions=" << to_string(partitions);
   out << ", " << "environmentContext=" << to_string(environmentContext);
   out << ", " << "txnId="; (__isset.txnId ? (out << to_string(txnId)) : (out << "<null>"));
+  out << ", " << "writeId="; (__isset.writeId ? (out << to_string(writeId)) : (out << "<null>"));
   out << ", " << "validWriteIdList="; (__isset.validWriteIdList ? (out << to_string(validWriteIdList)) : (out << "<null>"));
   out << ")";
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/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 68e34d5..81eadbc 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
@@ -3115,7 +3115,7 @@ inline std::ostream& operator<<(std::ostream& out, const StorageDescriptor& obj)
 }
 
 typedef struct _Table__isset {
-  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false), creationMetadata(false), catName(false), ownerType(true), txnId(true), validWriteIdList(false), isStatsCompliant(false) {}
+  _Table__isset() : tableName(false), dbName(false), owner(false), createTime(false), lastAccessTime(false), retention(false), sd(false), partitionKeys(false), parameters(false), viewOriginalText(false), viewExpandedText(false), tableType(false), privileges(false), temporary(true), rewriteEnabled(false), creationMetadata(false), catName(false), ownerType(true), txnId(true), writeId(true), validWriteIdList(false), isStatsCompliant(false) {}
   bool tableName :1;
   bool dbName :1;
   bool owner :1;
@@ -3135,6 +3135,7 @@ typedef struct _Table__isset {
   bool catName :1;
   bool ownerType :1;
   bool txnId :1;
+  bool writeId :1;
   bool validWriteIdList :1;
   bool isStatsCompliant :1;
 } _Table__isset;
@@ -3144,7 +3145,7 @@ class Table {
 
   Table(const Table&);
   Table& operator=(const Table&);
-  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0), catName(), ownerType((PrincipalType::type)1), txnId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
+  Table() : tableName(), dbName(), owner(), createTime(0), lastAccessTime(0), retention(0), viewOriginalText(), viewExpandedText(), tableType(), temporary(false), rewriteEnabled(0), catName(), ownerType((PrincipalType::type)1), txnId(-1LL), writeId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
     ownerType = (PrincipalType::type)1;
 
   }
@@ -3169,6 +3170,7 @@ class Table {
   std::string catName;
   PrincipalType::type ownerType;
   int64_t txnId;
+  int64_t writeId;
   std::string validWriteIdList;
   IsolationLevelCompliance::type isStatsCompliant;
 
@@ -3212,6 +3214,8 @@ class Table {
 
   void __set_txnId(const int64_t val);
 
+  void __set_writeId(const int64_t val);
+
   void __set_validWriteIdList(const std::string& val);
 
   void __set_isStatsCompliant(const IsolationLevelCompliance::type val);
@@ -3270,6 +3274,10 @@ class Table {
       return false;
     else if (__isset.txnId && !(txnId == rhs.txnId))
       return false;
+    if (__isset.writeId != rhs.__isset.writeId)
+      return false;
+    else if (__isset.writeId && !(writeId == rhs.writeId))
+      return false;
     if (__isset.validWriteIdList != rhs.__isset.validWriteIdList)
       return false;
     else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList))
@@ -3301,7 +3309,7 @@ inline std::ostream& operator<<(std::ostream& out, const Table& obj)
 }
 
 typedef struct _Partition__isset {
-  _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false), catName(false), txnId(true), validWriteIdList(false), isStatsCompliant(false) {}
+  _Partition__isset() : values(false), dbName(false), tableName(false), createTime(false), lastAccessTime(false), sd(false), parameters(false), privileges(false), catName(false), txnId(true), writeId(true), validWriteIdList(false), isStatsCompliant(false) {}
   bool values :1;
   bool dbName :1;
   bool tableName :1;
@@ -3312,6 +3320,7 @@ typedef struct _Partition__isset {
   bool privileges :1;
   bool catName :1;
   bool txnId :1;
+  bool writeId :1;
   bool validWriteIdList :1;
   bool isStatsCompliant :1;
 } _Partition__isset;
@@ -3321,7 +3330,7 @@ class Partition {
 
   Partition(const Partition&);
   Partition& operator=(const Partition&);
-  Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0), catName(), txnId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
+  Partition() : dbName(), tableName(), createTime(0), lastAccessTime(0), catName(), txnId(-1LL), writeId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
   }
 
   virtual ~Partition() throw();
@@ -3335,6 +3344,7 @@ class Partition {
   PrincipalPrivilegeSet privileges;
   std::string catName;
   int64_t txnId;
+  int64_t writeId;
   std::string validWriteIdList;
   IsolationLevelCompliance::type isStatsCompliant;
 
@@ -3360,6 +3370,8 @@ class Partition {
 
   void __set_txnId(const int64_t val);
 
+  void __set_writeId(const int64_t val);
+
   void __set_validWriteIdList(const std::string& val);
 
   void __set_isStatsCompliant(const IsolationLevelCompliance::type val);
@@ -3392,6 +3404,10 @@ class Partition {
       return false;
     else if (__isset.txnId && !(txnId == rhs.txnId))
       return false;
+    if (__isset.writeId != rhs.__isset.writeId)
+      return false;
+    else if (__isset.writeId && !(writeId == rhs.writeId))
+      return false;
     if (__isset.validWriteIdList != rhs.__isset.validWriteIdList)
       return false;
     else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList))
@@ -3599,7 +3615,7 @@ inline std::ostream& operator<<(std::ostream& out, const PartitionListComposingS
 }
 
 typedef struct _PartitionSpec__isset {
-  _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false), catName(false), txnId(true), validWriteIdList(false), isStatsCompliant(false) {}
+  _PartitionSpec__isset() : dbName(false), tableName(false), rootPath(false), sharedSDPartitionSpec(false), partitionList(false), catName(false), txnId(true), writeId(true), validWriteIdList(false), isStatsCompliant(false) {}
   bool dbName :1;
   bool tableName :1;
   bool rootPath :1;
@@ -3607,6 +3623,7 @@ typedef struct _PartitionSpec__isset {
   bool partitionList :1;
   bool catName :1;
   bool txnId :1;
+  bool writeId :1;
   bool validWriteIdList :1;
   bool isStatsCompliant :1;
 } _PartitionSpec__isset;
@@ -3616,7 +3633,7 @@ class PartitionSpec {
 
   PartitionSpec(const PartitionSpec&);
   PartitionSpec& operator=(const PartitionSpec&);
-  PartitionSpec() : dbName(), tableName(), rootPath(), catName(), txnId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
+  PartitionSpec() : dbName(), tableName(), rootPath(), catName(), txnId(-1LL), writeId(-1LL), validWriteIdList(), isStatsCompliant((IsolationLevelCompliance::type)0) {
   }
 
   virtual ~PartitionSpec() throw();
@@ -3627,6 +3644,7 @@ class PartitionSpec {
   PartitionListComposingSpec partitionList;
   std::string catName;
   int64_t txnId;
+  int64_t writeId;
   std::string validWriteIdList;
   IsolationLevelCompliance::type isStatsCompliant;
 
@@ -3646,6 +3664,8 @@ class PartitionSpec {
 
   void __set_txnId(const int64_t val);
 
+  void __set_writeId(const int64_t val);
+
   void __set_validWriteIdList(const std::string& val);
 
   void __set_isStatsCompliant(const IsolationLevelCompliance::type val);
@@ -3674,6 +3694,10 @@ class PartitionSpec {
       return false;
     else if (__isset.txnId && !(txnId == rhs.txnId))
       return false;
+    if (__isset.writeId != rhs.__isset.writeId)
+      return false;
+    else if (__isset.writeId && !(writeId == rhs.writeId))
+      return false;
     if (__isset.validWriteIdList != rhs.__isset.validWriteIdList)
       return false;
     else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList))
@@ -4623,9 +4647,10 @@ inline std::ostream& operator<<(std::ostream& out, const AggrStats& obj)
 }
 
 typedef struct _SetPartitionsStatsRequest__isset {
-  _SetPartitionsStatsRequest__isset() : needMerge(false), txnId(true), validWriteIdList(false) {}
+  _SetPartitionsStatsRequest__isset() : needMerge(false), txnId(true), writeId(true), validWriteIdList(false) {}
   bool needMerge :1;
   bool txnId :1;
+  bool writeId :1;
   bool validWriteIdList :1;
 } _SetPartitionsStatsRequest__isset;
 
@@ -4634,13 +4659,14 @@ class SetPartitionsStatsRequest {
 
   SetPartitionsStatsRequest(const SetPartitionsStatsRequest&);
   SetPartitionsStatsRequest& operator=(const SetPartitionsStatsRequest&);
-  SetPartitionsStatsRequest() : needMerge(0), txnId(-1LL), validWriteIdList() {
+  SetPartitionsStatsRequest() : needMerge(0), txnId(-1LL), writeId(-1LL), validWriteIdList() {
   }
 
   virtual ~SetPartitionsStatsRequest() throw();
   std::vector<ColumnStatistics>  colStats;
   bool needMerge;
   int64_t txnId;
+  int64_t writeId;
   std::string validWriteIdList;
 
   _SetPartitionsStatsRequest__isset __isset;
@@ -4651,6 +4677,8 @@ class SetPartitionsStatsRequest {
 
   void __set_txnId(const int64_t val);
 
+  void __set_writeId(const int64_t val);
+
   void __set_validWriteIdList(const std::string& val);
 
   bool operator == (const SetPartitionsStatsRequest & rhs) const
@@ -4665,6 +4693,10 @@ class SetPartitionsStatsRequest {
       return false;
     else if (__isset.txnId && !(txnId == rhs.txnId))
       return false;
+    if (__isset.writeId != rhs.__isset.writeId)
+      return false;
+    else if (__isset.writeId && !(writeId == rhs.writeId))
+      return false;
     if (__isset.validWriteIdList != rhs.__isset.validWriteIdList)
       return false;
     else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList))
@@ -13442,8 +13474,9 @@ inline std::ostream& operator<<(std::ostream& out, const GetRuntimeStatsRequest&
 }
 
 typedef struct _AlterPartitionsRequest__isset {
-  _AlterPartitionsRequest__isset() : txnId(true), validWriteIdList(false) {}
+  _AlterPartitionsRequest__isset() : txnId(true), writeId(true), validWriteIdList(false) {}
   bool txnId :1;
+  bool writeId :1;
   bool validWriteIdList :1;
 } _AlterPartitionsRequest__isset;
 
@@ -13452,7 +13485,7 @@ class AlterPartitionsRequest {
 
   AlterPartitionsRequest(const AlterPartitionsRequest&);
   AlterPartitionsRequest& operator=(const AlterPartitionsRequest&);
-  AlterPartitionsRequest() : dbName(), tableName(), txnId(-1LL), validWriteIdList() {
+  AlterPartitionsRequest() : dbName(), tableName(), txnId(-1LL), writeId(-1LL), validWriteIdList() {
   }
 
   virtual ~AlterPartitionsRequest() throw();
@@ -13461,6 +13494,7 @@ class AlterPartitionsRequest {
   std::vector<Partition>  partitions;
   EnvironmentContext environmentContext;
   int64_t txnId;
+  int64_t writeId;
   std::string validWriteIdList;
 
   _AlterPartitionsRequest__isset __isset;
@@ -13475,6 +13509,8 @@ class AlterPartitionsRequest {
 
   void __set_txnId(const int64_t val);
 
+  void __set_writeId(const int64_t val);
+
   void __set_validWriteIdList(const std::string& val);
 
   bool operator == (const AlterPartitionsRequest & rhs) const
@@ -13491,6 +13527,10 @@ class AlterPartitionsRequest {
       return false;
     else if (__isset.txnId && !(txnId == rhs.txnId))
       return false;
+    if (__isset.writeId != rhs.__isset.writeId)
+      return false;
+    else if (__isset.writeId && !(writeId == rhs.writeId))
+      return false;
     if (__isset.validWriteIdList != rhs.__isset.validWriteIdList)
       return false;
     else if (__isset.validWriteIdList && !(validWriteIdList == rhs.validWriteIdList))

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
index 8d4102f..314ea90 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
@@ -43,7 +43,8 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)3);
   private static final org.apache.thrift.protocol.TField ENVIRONMENT_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("environmentContext", org.apache.thrift.protocol.TType.STRUCT, (short)4);
   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)5);
-  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)6);
+  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)6);
+  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)7);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -56,6 +57,7 @@ import org.slf4j.LoggerFactory;
   private List<Partition> partitions; // required
   private EnvironmentContext environmentContext; // required
   private long txnId; // optional
+  private long writeId; // optional
   private String validWriteIdList; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -65,7 +67,8 @@ import org.slf4j.LoggerFactory;
     PARTITIONS((short)3, "partitions"),
     ENVIRONMENT_CONTEXT((short)4, "environmentContext"),
     TXN_ID((short)5, "txnId"),
-    VALID_WRITE_ID_LIST((short)6, "validWriteIdList");
+    WRITE_ID((short)6, "writeId"),
+    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -90,7 +93,9 @@ import org.slf4j.LoggerFactory;
           return ENVIRONMENT_CONTEXT;
         case 5: // TXN_ID
           return TXN_ID;
-        case 6: // VALID_WRITE_ID_LIST
+        case 6: // WRITE_ID
+          return WRITE_ID;
+        case 7: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
         default:
           return null;
@@ -133,8 +138,9 @@ import org.slf4j.LoggerFactory;
 
   // 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.TXN_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_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);
@@ -149,6 +155,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, EnvironmentContext.class)));
     tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
@@ -158,6 +166,8 @@ import org.slf4j.LoggerFactory;
   public AlterPartitionsRequest() {
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
   }
 
   public AlterPartitionsRequest(
@@ -195,6 +205,7 @@ import org.slf4j.LoggerFactory;
       this.environmentContext = new EnvironmentContext(other.environmentContext);
     }
     this.txnId = other.txnId;
+    this.writeId = other.writeId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
@@ -212,6 +223,8 @@ import org.slf4j.LoggerFactory;
     this.environmentContext = null;
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
     this.validWriteIdList = null;
   }
 
@@ -344,6 +357,28 @@ import org.slf4j.LoggerFactory;
     __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 getValidWriteIdList() {
     return this.validWriteIdList;
   }
@@ -409,6 +444,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
     case VALID_WRITE_ID_LIST:
       if (value == null) {
         unsetValidWriteIdList();
@@ -437,6 +480,9 @@ import org.slf4j.LoggerFactory;
     case TXN_ID:
       return getTxnId();
 
+    case WRITE_ID:
+      return getWriteId();
+
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
@@ -461,6 +507,8 @@ import org.slf4j.LoggerFactory;
       return isSetEnvironmentContext();
     case TXN_ID:
       return isSetTxnId();
+    case WRITE_ID:
+      return isSetWriteId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
     }
@@ -525,6 +573,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_writeId = true && this.isSetWriteId();
+    boolean that_present_writeId = true && that.isSetWriteId();
+    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_validWriteIdList = true && this.isSetValidWriteIdList();
     boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
     if (this_present_validWriteIdList || that_present_validWriteIdList) {
@@ -566,6 +623,11 @@ import org.slf4j.LoggerFactory;
     if (present_txnId)
       list.add(txnId);
 
+    boolean present_writeId = true && (isSetWriteId());
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
     boolean present_validWriteIdList = true && (isSetValidWriteIdList());
     list.add(present_validWriteIdList);
     if (present_validWriteIdList)
@@ -632,6 +694,16 @@ import org.slf4j.LoggerFactory;
         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(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
@@ -699,6 +771,12 @@ import org.slf4j.LoggerFactory;
       sb.append(this.txnId);
       first = false;
     }
+    if (isSetWriteId()) {
+      if (!first) sb.append(", ");
+      sb.append("writeId:");
+      sb.append(this.writeId);
+      first = false;
+    }
     if (isSetValidWriteIdList()) {
       if (!first) sb.append(", ");
       sb.append("validWriteIdList:");
@@ -825,7 +903,15 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // VALID_WRITE_ID_LIST
+          case 6: // 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 7: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -878,6 +964,11 @@ import org.slf4j.LoggerFactory;
         oprot.writeI64(struct.txnId);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+        oprot.writeI64(struct.writeId);
+        oprot.writeFieldEnd();
+      }
       if (struct.validWriteIdList != null) {
         if (struct.isSetValidWriteIdList()) {
           oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
@@ -916,13 +1007,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnId()) {
         optionals.set(0);
       }
-      if (struct.isSetValidWriteIdList()) {
+      if (struct.isSetWriteId()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetValidWriteIdList()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetTxnId()) {
         oprot.writeI64(struct.txnId);
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeI64(struct.writeId);
+      }
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
@@ -950,12 +1047,16 @@ import org.slf4j.LoggerFactory;
       struct.environmentContext = new EnvironmentContext();
       struct.environmentContext.read(iprot);
       struct.setEnvironmentContextIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.txnId = iprot.readI64();
         struct.setTxnIdIsSet(true);
       }
       if (incoming.get(1)) {
+        struct.writeId = iprot.readI64();
+        struct.setWriteIdIsSet(true);
+      }
+      if (incoming.get(2)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
index 5b40d2f..37515a0 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Partition.java
@@ -48,8 +48,9 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PRIVILEGES_FIELD_DESC = new org.apache.thrift.protocol.TField("privileges", org.apache.thrift.protocol.TType.STRUCT, (short)8);
   private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)9);
   private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnId", org.apache.thrift.protocol.TType.I64, (short)10);
-  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)11);
-  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.I32, (short)12);
+  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)11);
+  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)12);
+  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.I32, (short)13);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -67,6 +68,7 @@ import org.slf4j.LoggerFactory;
   private PrincipalPrivilegeSet privileges; // optional
   private String catName; // optional
   private long txnId; // optional
+  private long writeId; // optional
   private String validWriteIdList; // optional
   private IsolationLevelCompliance isStatsCompliant; // optional
 
@@ -82,12 +84,13 @@ import org.slf4j.LoggerFactory;
     PRIVILEGES((short)8, "privileges"),
     CAT_NAME((short)9, "catName"),
     TXN_ID((short)10, "txnId"),
-    VALID_WRITE_ID_LIST((short)11, "validWriteIdList"),
+    WRITE_ID((short)11, "writeId"),
+    VALID_WRITE_ID_LIST((short)12, "validWriteIdList"),
     /**
      * 
      * @see IsolationLevelCompliance
      */
-    IS_STATS_COMPLIANT((short)12, "isStatsCompliant");
+    IS_STATS_COMPLIANT((short)13, "isStatsCompliant");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -122,9 +125,11 @@ import org.slf4j.LoggerFactory;
           return CAT_NAME;
         case 10: // TXN_ID
           return TXN_ID;
-        case 11: // VALID_WRITE_ID_LIST
+        case 11: // WRITE_ID
+          return WRITE_ID;
+        case 12: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
-        case 12: // IS_STATS_COMPLIANT
+        case 13: // IS_STATS_COMPLIANT
           return IS_STATS_COMPLIANT;
         default:
           return null;
@@ -169,8 +174,9 @@ import org.slf4j.LoggerFactory;
   private static final int __CREATETIME_ISSET_ID = 0;
   private static final int __LASTACCESSTIME_ISSET_ID = 1;
   private static final int __TXNID_ISSET_ID = 2;
+  private static final int __WRITEID_ISSET_ID = 3;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.IS_STATS_COMPLIANT};
+  private static final _Fields optionals[] = {_Fields.PRIVILEGES,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.IS_STATS_COMPLIANT};
   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);
@@ -197,6 +203,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.IS_STATS_COMPLIANT, new org.apache.thrift.meta_data.FieldMetaData("isStatsCompliant", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -208,6 +216,8 @@ import org.slf4j.LoggerFactory;
   public Partition() {
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
   }
 
   public Partition(
@@ -262,6 +272,7 @@ import org.slf4j.LoggerFactory;
       this.catName = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(other.catName);
     }
     this.txnId = other.txnId;
+    this.writeId = other.writeId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
@@ -289,6 +300,8 @@ import org.slf4j.LoggerFactory;
     this.catName = null;
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
     this.validWriteIdList = null;
     this.isStatsCompliant = null;
   }
@@ -546,6 +559,28 @@ import org.slf4j.LoggerFactory;
     __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 getValidWriteIdList() {
     return this.validWriteIdList;
   }
@@ -682,6 +717,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
     case VALID_WRITE_ID_LIST:
       if (value == null) {
         unsetValidWriteIdList();
@@ -733,6 +776,9 @@ import org.slf4j.LoggerFactory;
     case TXN_ID:
       return getTxnId();
 
+    case WRITE_ID:
+      return getWriteId();
+
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
@@ -770,6 +816,8 @@ import org.slf4j.LoggerFactory;
       return isSetCatName();
     case TXN_ID:
       return isSetTxnId();
+    case WRITE_ID:
+      return isSetWriteId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
     case IS_STATS_COMPLIANT:
@@ -881,6 +929,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_writeId = true && this.isSetWriteId();
+    boolean that_present_writeId = true && that.isSetWriteId();
+    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_validWriteIdList = true && this.isSetValidWriteIdList();
     boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
     if (this_present_validWriteIdList || that_present_validWriteIdList) {
@@ -956,6 +1013,11 @@ import org.slf4j.LoggerFactory;
     if (present_txnId)
       list.add(txnId);
 
+    boolean present_writeId = true && (isSetWriteId());
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
     boolean present_validWriteIdList = true && (isSetValidWriteIdList());
     list.add(present_validWriteIdList);
     if (present_validWriteIdList)
@@ -1077,6 +1139,16 @@ import org.slf4j.LoggerFactory;
         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(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
@@ -1190,6 +1262,12 @@ import org.slf4j.LoggerFactory;
       sb.append(this.txnId);
       first = false;
     }
+    if (isSetWriteId()) {
+      if (!first) sb.append(", ");
+      sb.append("writeId:");
+      sb.append(this.writeId);
+      first = false;
+    }
     if (isSetValidWriteIdList()) {
       if (!first) sb.append(", ");
       sb.append("validWriteIdList:");
@@ -1365,7 +1443,15 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 11: // VALID_WRITE_ID_LIST
+          case 11: // 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 12: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -1373,7 +1459,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 12: // IS_STATS_COMPLIANT
+          case 13: // IS_STATS_COMPLIANT
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.isStatsCompliant = org.apache.hadoop.hive.metastore.api.IsolationLevelCompliance.findByValue(iprot.readI32());
               struct.setIsStatsCompliantIsSet(true);
@@ -1459,6 +1545,11 @@ import org.slf4j.LoggerFactory;
         oprot.writeI64(struct.txnId);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+        oprot.writeI64(struct.writeId);
+        oprot.writeFieldEnd();
+      }
       if (struct.validWriteIdList != null) {
         if (struct.isSetValidWriteIdList()) {
           oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
@@ -1521,13 +1612,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnId()) {
         optionals.set(9);
       }
-      if (struct.isSetValidWriteIdList()) {
+      if (struct.isSetWriteId()) {
         optionals.set(10);
       }
-      if (struct.isSetIsStatsCompliant()) {
+      if (struct.isSetValidWriteIdList()) {
         optionals.set(11);
       }
-      oprot.writeBitSet(optionals, 12);
+      if (struct.isSetIsStatsCompliant()) {
+        optionals.set(12);
+      }
+      oprot.writeBitSet(optionals, 13);
       if (struct.isSetValues()) {
         {
           oprot.writeI32(struct.values.size());
@@ -1571,6 +1665,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnId()) {
         oprot.writeI64(struct.txnId);
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeI64(struct.writeId);
+      }
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
@@ -1582,7 +1679,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Partition struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(12);
+      BitSet incoming = iprot.readBitSet(13);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TList _list227 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
@@ -1646,10 +1743,14 @@ import org.slf4j.LoggerFactory;
         struct.setTxnIdIsSet(true);
       }
       if (incoming.get(10)) {
+        struct.writeId = iprot.readI64();
+        struct.setWriteIdIsSet(true);
+      }
+      if (incoming.get(11)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }
-      if (incoming.get(11)) {
+      if (incoming.get(12)) {
         struct.isStatsCompliant = org.apache.hadoop.hive.metastore.api.IsolationLevelCompliance.findByValue(iprot.readI32());
         struct.setIsStatsCompliantIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/222b4aca/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
index bc625b0..63b88db 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionSpec.java
@@ -45,8 +45,9 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField PARTITION_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionList", org.apache.thrift.protocol.TType.STRUCT, (short)5);
   private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)6);
   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)7);
-  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)8);
-  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.I32, (short)9);
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)8);
+  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)9);
+  private static final org.apache.thrift.protocol.TField IS_STATS_COMPLIANT_FIELD_DESC = new org.apache.thrift.protocol.TField("isStatsCompliant", org.apache.thrift.protocol.TType.I32, (short)10);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -61,6 +62,7 @@ import org.slf4j.LoggerFactory;
   private PartitionListComposingSpec partitionList; // optional
   private String catName; // optional
   private long txnId; // optional
+  private long writeId; // optional
   private String validWriteIdList; // optional
   private IsolationLevelCompliance isStatsCompliant; // optional
 
@@ -73,12 +75,13 @@ import org.slf4j.LoggerFactory;
     PARTITION_LIST((short)5, "partitionList"),
     CAT_NAME((short)6, "catName"),
     TXN_ID((short)7, "txnId"),
-    VALID_WRITE_ID_LIST((short)8, "validWriteIdList"),
+    WRITE_ID((short)8, "writeId"),
+    VALID_WRITE_ID_LIST((short)9, "validWriteIdList"),
     /**
      * 
      * @see IsolationLevelCompliance
      */
-    IS_STATS_COMPLIANT((short)9, "isStatsCompliant");
+    IS_STATS_COMPLIANT((short)10, "isStatsCompliant");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -107,9 +110,11 @@ import org.slf4j.LoggerFactory;
           return CAT_NAME;
         case 7: // TXN_ID
           return TXN_ID;
-        case 8: // VALID_WRITE_ID_LIST
+        case 8: // WRITE_ID
+          return WRITE_ID;
+        case 9: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
-        case 9: // IS_STATS_COMPLIANT
+        case 10: // IS_STATS_COMPLIANT
           return IS_STATS_COMPLIANT;
         default:
           return null;
@@ -152,8 +157,9 @@ import org.slf4j.LoggerFactory;
 
   // 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.SHARED_SDPARTITION_SPEC,_Fields.PARTITION_LIST,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.IS_STATS_COMPLIANT};
+  private static final _Fields optionals[] = {_Fields.SHARED_SDPARTITION_SPEC,_Fields.PARTITION_LIST,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST,_Fields.IS_STATS_COMPLIANT};
   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);
@@ -171,6 +177,8 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.IS_STATS_COMPLIANT, new org.apache.thrift.meta_data.FieldMetaData("isStatsCompliant", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -182,6 +190,8 @@ import org.slf4j.LoggerFactory;
   public PartitionSpec() {
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
   }
 
   public PartitionSpec(
@@ -219,6 +229,7 @@ import org.slf4j.LoggerFactory;
       this.catName = other.catName;
     }
     this.txnId = other.txnId;
+    this.writeId = other.writeId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
@@ -241,6 +252,8 @@ import org.slf4j.LoggerFactory;
     this.catName = null;
     this.txnId = -1L;
 
+    this.writeId = -1L;
+
     this.validWriteIdList = null;
     this.isStatsCompliant = null;
   }
@@ -405,6 +418,28 @@ import org.slf4j.LoggerFactory;
     __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 getValidWriteIdList() {
     return this.validWriteIdList;
   }
@@ -517,6 +552,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
     case VALID_WRITE_ID_LIST:
       if (value == null) {
         unsetValidWriteIdList();
@@ -559,6 +602,9 @@ import org.slf4j.LoggerFactory;
     case TXN_ID:
       return getTxnId();
 
+    case WRITE_ID:
+      return getWriteId();
+
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
@@ -590,6 +636,8 @@ import org.slf4j.LoggerFactory;
       return isSetCatName();
     case TXN_ID:
       return isSetTxnId();
+    case WRITE_ID:
+      return isSetWriteId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
     case IS_STATS_COMPLIANT:
@@ -674,6 +722,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_writeId = true && this.isSetWriteId();
+    boolean that_present_writeId = true && that.isSetWriteId();
+    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_validWriteIdList = true && this.isSetValidWriteIdList();
     boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
     if (this_present_validWriteIdList || that_present_validWriteIdList) {
@@ -734,6 +791,11 @@ import org.slf4j.LoggerFactory;
     if (present_txnId)
       list.add(txnId);
 
+    boolean present_writeId = true && (isSetWriteId());
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
     boolean present_validWriteIdList = true && (isSetValidWriteIdList());
     list.add(present_validWriteIdList);
     if (present_validWriteIdList)
@@ -825,6 +887,16 @@ import org.slf4j.LoggerFactory;
         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(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
@@ -924,6 +996,12 @@ import org.slf4j.LoggerFactory;
       sb.append(this.txnId);
       first = false;
     }
+    if (isSetWriteId()) {
+      if (!first) sb.append(", ");
+      sb.append("writeId:");
+      sb.append(this.writeId);
+      first = false;
+    }
     if (isSetValidWriteIdList()) {
       if (!first) sb.append(", ");
       sb.append("validWriteIdList:");
@@ -1053,7 +1131,15 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // VALID_WRITE_ID_LIST
+          case 8: // 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 9: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -1061,7 +1147,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 9: // IS_STATS_COMPLIANT
+          case 10: // IS_STATS_COMPLIANT
             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
               struct.isStatsCompliant = org.apache.hadoop.hive.metastore.api.IsolationLevelCompliance.findByValue(iprot.readI32());
               struct.setIsStatsCompliantIsSet(true);
@@ -1123,6 +1209,11 @@ import org.slf4j.LoggerFactory;
         oprot.writeI64(struct.txnId);
         oprot.writeFieldEnd();
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+        oprot.writeI64(struct.writeId);
+        oprot.writeFieldEnd();
+      }
       if (struct.validWriteIdList != null) {
         if (struct.isSetValidWriteIdList()) {
           oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
@@ -1176,13 +1267,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnId()) {
         optionals.set(6);
       }
-      if (struct.isSetValidWriteIdList()) {
+      if (struct.isSetWriteId()) {
         optionals.set(7);
       }
-      if (struct.isSetIsStatsCompliant()) {
+      if (struct.isSetValidWriteIdList()) {
         optionals.set(8);
       }
-      oprot.writeBitSet(optionals, 9);
+      if (struct.isSetIsStatsCompliant()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
       if (struct.isSetDbName()) {
         oprot.writeString(struct.dbName);
       }
@@ -1204,6 +1298,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnId()) {
         oprot.writeI64(struct.txnId);
       }
+      if (struct.isSetWriteId()) {
+        oprot.writeI64(struct.writeId);
+      }
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
@@ -1215,7 +1312,7 @@ import org.slf4j.LoggerFactory;
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, PartitionSpec struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(9);
+      BitSet incoming = iprot.readBitSet(10);
       if (incoming.get(0)) {
         struct.dbName = iprot.readString();
         struct.setDbNameIsSet(true);
@@ -1247,10 +1344,14 @@ import org.slf4j.LoggerFactory;
         struct.setTxnIdIsSet(true);
       }
       if (incoming.get(7)) {
+        struct.writeId = iprot.readI64();
+        struct.setWriteIdIsSet(true);
+      }
+      if (incoming.get(8)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }
-      if (incoming.get(8)) {
+      if (incoming.get(9)) {
         struct.isStatsCompliant = org.apache.hadoop.hive.metastore.api.IsolationLevelCompliance.findByValue(iprot.readI32());
         struct.setIsStatsCompliantIsSet(true);
       }