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/07/19 22:58:10 UTC

[5/5] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)


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

Branch: refs/heads/master-txnstats
Commit: cdb32a7fbabc9baea535b94da159965eda4e23a8
Parents: 651e795
Author: sergey <se...@apache.org>
Authored: Thu Jul 19 15:48:39 2018 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Jul 19 15:48:39 2018 -0700

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |  37 ++-
 .../metastore/SynchronizedMetaStoreClient.java  |   4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  23 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  35 +--
 .../ql/metadata/SessionHiveMetaStoreClient.java |   9 +-
 .../hive/ql/optimizer/StatsOptimizer.java       |   4 -
 .../hadoop/hive/ql/stats/ColStatsProcessor.java |   1 -
 .../hive/ql/stats/StatsUpdaterThread.java       |   4 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |   2 +-
 .../hive/ql/stats/TestStatsUpdaterThread.java   |  34 +--
 .../metastore/api/AddPartitionsRequest.java     | 117 +--------
 .../metastore/api/AlterPartitionsRequest.java   | 131 ++--------
 .../hive/metastore/api/AlterTableRequest.java   | 131 ++--------
 .../hive/metastore/api/GetTableRequest.java     | 113 +--------
 .../metastore/api/PartitionsStatsRequest.java   | 121 +--------
 .../metastore/api/RenamePartitionRequest.java   | 121 +--------
 .../api/SetPartitionsStatsRequest.java          | 131 ++--------
 .../hive/metastore/api/TableStatsRequest.java   | 121 +--------
 .../metastore/api/TruncateTableRequest.java     | 131 ++--------
 .../src/gen/thrift/gen-php/metastore/Types.php  | 247 ++-----------------
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 192 +++-----------
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  42 +---
 .../src/main/thrift/hive_metastore.thrift       |  33 +--
 .../hadoop/hive/metastore/AlterHandler.java     |   8 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java |  52 ++--
 .../hadoop/hive/metastore/HiveMetaStore.java    | 133 +++++-----
 .../hive/metastore/HiveMetaStoreClient.java     |  63 ++---
 .../hadoop/hive/metastore/IHMSHandler.java      |   2 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |  30 +--
 .../hadoop/hive/metastore/ObjectStore.java      |  92 +++----
 .../apache/hadoop/hive/metastore/RawStore.java  |  20 +-
 .../hive/metastore/cache/CachedStore.java       |  56 ++---
 .../DummyRawStoreControlledCommit.java          |  38 +--
 .../DummyRawStoreForJdoConnection.java          |  22 +-
 .../HiveMetaStoreClientPreCatalog.java          |  41 +--
 .../InjectableBehaviourObjectStore.java         |   5 +-
 .../hive/metastore/TestHiveAlterHandler.java    |   6 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |   4 +-
 .../hadoop/hive/metastore/TestOldSchema.java    |   2 +-
 .../hive/metastore/cache/TestCachedStore.java   |  18 +-
 .../metastore/client/TestAlterPartitions.java   |   4 +-
 41 files changed, 506 insertions(+), 1874 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/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 270aa6c..1c105d1 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
@@ -268,8 +268,8 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public Table getTable(String catName, String dbName, String tableName,
-                        long txnId, String writeIdList) throws MetaException {
-    return objectStore.getTable(catName, dbName, tableName, txnId, writeIdList);
+                        String writeIdList) throws MetaException {
+    return objectStore.getTable(catName, dbName, tableName, writeIdList);
   }
 
   @Override
@@ -286,9 +286,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public Partition getPartition(String catName, String dbName, String tableName,
-                                List<String> partVals, long txnId, String writeIdList)
+                                List<String> partVals, String writeIdList)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getPartition(catName, dbName, tableName, partVals, txnId, writeIdList);
+    return objectStore.getPartition(catName, dbName, tableName, partVals, writeIdList);
   }
 
   @Override
@@ -321,10 +321,10 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public void alterTable(String catName, String dbName, String name, Table newTable, long queryTxnId, String queryValidWriteIds)
+  public void alterTable(String catName, String dbName, String name, Table newTable, String queryValidWriteIds)
       throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
-      objectStore.alterTable(catName, dbName, name, newTable, queryTxnId, queryValidWriteIds);
+      objectStore.alterTable(catName, dbName, name, newTable, queryValidWriteIds);
     } else {
       throw new RuntimeException("Event failed.");
     }
@@ -386,9 +386,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public void alterPartition(String catName, String dbName, String tblName, List<String> partVals,
-                             Partition newPart, long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+                             Partition newPart, String queryValidWriteIds) throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
-      objectStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryTxnId, queryValidWriteIds);
+      objectStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryValidWriteIds);
     } else {
       throw new RuntimeException("Event failed.");
     }
@@ -397,10 +397,10 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public void alterPartitions(String catName, String dbName, String tblName,
                               List<List<String>> partValsList, List<Partition> newParts,
-                              long writeId, long queryTxnId, String queryValidWriteIds)
+                              long writeId, String queryValidWriteIds)
       throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
-      objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts, writeId, queryTxnId, queryValidWriteIds);
+      objectStore.alterPartitions(catName, dbName, tblName, partValsList, newParts, writeId, queryValidWriteIds);
     } else {
       throw new RuntimeException("Event failed.");
     }
@@ -714,9 +714,9 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tableName,
                                                    List<String> colNames,
-                                                   long txnId, String writeIdList)
+                                                   String writeIdList)
       throws MetaException, NoSuchObjectException {
-    return objectStore.getTableColumnStatistics(catName, dbName, tableName, colNames, txnId, writeIdList);
+    return objectStore.getTableColumnStatistics(catName, dbName, tableName, colNames, writeIdList);
   }
 
   @Override
@@ -736,16 +736,16 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public boolean updateTableColumnStatistics(ColumnStatistics statsObj, long txnId, String validWriteIds, long writeId)
+  public boolean updateTableColumnStatistics(ColumnStatistics statsObj, String validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-    return objectStore.updateTableColumnStatistics(statsObj, txnId, validWriteIds, writeId);
+    return objectStore.updateTableColumnStatistics(statsObj, validWriteIds, writeId);
   }
 
   @Override
   public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj,
-      List<String> partVals, long txnId, String validWriteIds, long writeId)
+      List<String> partVals, String validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-    return objectStore.updatePartitionColumnStatistics(statsObj, partVals, txnId, validWriteIds, writeId);
+    return objectStore.updatePartitionColumnStatistics(statsObj, partVals, validWriteIds, writeId);
   }
 
   @Override
@@ -814,11 +814,10 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   public List<ColumnStatistics> getPartitionColumnStatistics(String catName, String dbName,
                                                              String tblName, List<String> colNames,
                                                              List<String> partNames,
-                                                             long txnId,
                                                              String writeIdList)
       throws MetaException, NoSuchObjectException {
     return objectStore.getPartitionColumnStatistics(
-        catName, dbName, tblName  , colNames, partNames, txnId, writeIdList);
+        catName, dbName, tblName  , colNames, partNames, writeIdList);
   }
 
   @Override
@@ -901,7 +900,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public AggrStats get_aggr_stats_for(String catName, String dbName,
                                       String tblName, List<String> partNames, List<String> colNames,
-                                      long txnId, String writeIdList)
+                                      String writeIdList)
       throws MetaException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/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 7eddc16..1e279f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
@@ -79,8 +79,8 @@ public final class SynchronizedMetaStoreClient {
   }
 
   public synchronized void alter_partition(String dbName, String tblName,
-      Partition newPart, EnvironmentContext environmentContext, long txnId, String writeIdList) throws TException {
-    client.alter_partition(dbName, tblName, newPart, environmentContext, txnId, writeIdList);
+      Partition newPart, EnvironmentContext environmentContext, String writeIdList) throws TException {
+    client.alter_partition(dbName, tblName, newPart, environmentContext, writeIdList);
   }
 
   public synchronized LockResponse checkLock(long lockid) throws TException {

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/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 9b66bcf..183d690 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
@@ -1610,31 +1610,21 @@ public class AcidUtils {
   }
 
   public static class TableSnapshot {
-    private long txnId;
     private long writeId;
     private String validWriteIdList;
 
     public TableSnapshot() {
     }
 
-    public TableSnapshot(long txnId, long writeId, String validWriteIdList) {
-      this.txnId = txnId;
+    public TableSnapshot(long writeId, String validWriteIdList) {
       this.writeId = writeId;
       this.validWriteIdList = validWriteIdList;
     }
 
-    public long getTxnId() {
-      return txnId;
-    }
-
     public String getValidWriteIdList() {
       return validWriteIdList;
     }
 
-    public void setTxnId(long txnId) {
-      this.txnId = txnId;
-    }
-
     public long getWriteId() {
       return writeId;
     }
@@ -1649,7 +1639,7 @@ public class AcidUtils {
 
     @Override
     public String toString() {
-      return "[txnId=" + txnId + ", validWriteIdList=" + validWriteIdList + ", writeId=" + writeId + "]";
+      return "[validWriteIdList=" + validWriteIdList + ", writeId=" + writeId + "]";
     }
   }
 
@@ -1681,17 +1671,12 @@ public class AcidUtils {
     if (tblName == null) {
       tblName = tbl.getTableName();
     }
-    long txnId = -1;
     long writeId = -1;
     ValidWriteIdList validWriteIdList = null;
 
     HiveTxnManager sessionTxnMgr = SessionState.get().getTxnMgr();
-
-    if (sessionTxnMgr != null) {
-      txnId = sessionTxnMgr.getCurrentTxnId();
-    }
     String fullTableName = getFullTableName(dbName, tblName);
-    if (txnId > 0) {
+    if (sessionTxnMgr != null && sessionTxnMgr.getCurrentTxnId() > 0) {
       validWriteIdList = getTableValidWriteIdList(conf, fullTableName);
       if (isStatsUpdater) {
         writeId = SessionState.get().getTxnMgr() != null ?
@@ -1718,7 +1703,7 @@ public class AcidUtils {
         throw new AssertionError("Cannot find valid write ID list for " + tblName);
       }
     }
-    return new TableSnapshot(txnId, writeId,
+    return new TableSnapshot(writeId,
         validWriteIdList != null ? validWriteIdList.toString() : null);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/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 a9accbf..addbd6d 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
@@ -652,7 +652,6 @@ public class Hive {
       // Why is alter_partitions synchronized while this isn't?
       getMSC().alter_table(
           catName, dbName, tblName, newTbl.getTTable(), environmentContext,
-          tableSnapshot == null ? -1 : tableSnapshot.getTxnId(),
           tableSnapshot == null ? null : tableSnapshot.getValidWriteIdList());
     } catch (MetaException e) {
       throw new HiveException("Unable to alter table. " + e.getMessage(), e);
@@ -729,7 +728,6 @@ public class Hive {
       }
       getSynchronizedMSC().alter_partition(
           dbName, tblName, newPart.getTPartition(), environmentContext,
-          tableSnapshot == null ? -1 : tableSnapshot.getTxnId(),
           tableSnapshot == null ? null : tableSnapshot.getValidWriteIdList());
 
     } catch (MetaException e) {
@@ -784,9 +782,8 @@ public class Hive {
         newTParts.add(tmpPart.getTPartition());
       }
       getMSC().alter_partitions(names[0], names[1], newTParts, environmentContext,
-          tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
           tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null,
-          tableSnapshot != null ? tableSnapshot.getWriteId() : -1    );
+          tableSnapshot != null ? tableSnapshot.getWriteId() : -1);
     } catch (MetaException e) {
       throw new HiveException("Unable to alter partition. " + e.getMessage(), e);
     } catch (TException e) {
@@ -829,19 +826,17 @@ public class Hive {
         }
       }
       String validWriteIds = null;
-      long txnId = -1;
       if (AcidUtils.isTransactionalTable(tbl)) {
         // Set table snapshot to api.Table to make it persistent.
         TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl, true);
         if (tableSnapshot != null) {
           newPart.getTPartition().setWriteId(tableSnapshot.getWriteId());
-          txnId = tableSnapshot.getTxnId();
           validWriteIds = tableSnapshot.getValidWriteIdList();
         }
       }
 
       getMSC().renamePartition(tbl.getCatName(), tbl.getDbName(), tbl.getTableName(), pvals,
-          newPart.getTPartition(), txnId, validWriteIds);
+          newPart.getTPartition(), validWriteIds);
 
     } catch (InvalidOperationException e){
       throw new HiveException("Unable to rename partition. " + e.getMessage(), e);
@@ -1079,7 +1074,7 @@ public class Hive {
         getMSC().truncateTable(table.getDbName(), table.getTableName(), partNames);
       } else {
         getMSC().truncateTable(table.getDbName(), table.getTableName(), partNames,
-            snapshot.getTxnId(), snapshot.getValidWriteIdList(), snapshot.getWriteId());
+            snapshot.getValidWriteIdList(), snapshot.getWriteId());
       }
     } catch (Exception e) {
       throw new HiveException(e);
@@ -1185,7 +1180,7 @@ public class Hive {
           validWriteIdList = AcidUtils.getTableValidWriteIdListWithTxnList(conf,
               dbName, tableName);
         }
-        tTable = getMSC().getTable(dbName, tableName, txnId,
+        tTable = getMSC().getTable(getDefaultCatalog(conf), dbName, tableName,
             validWriteIdList != null ? validWriteIdList.toString() : null);
       } else {
         tTable = getMSC().getTable(dbName, tableName);
@@ -2092,7 +2087,6 @@ public class Hive {
     LOG.debug("Altering existing partition " + newTPart.getSpec());
     getSynchronizedMSC().alter_partition(
         tbl.getDbName(), tbl.getTableName(), newTPart.getTPartition(), new EnvironmentContext(),
-        tableSnapshot == null ? -1 : tableSnapshot.getTxnId(),
         tableSnapshot == null ? null : tableSnapshot.getValidWriteIdList());
   }
 
@@ -2614,7 +2608,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
           out.add(new Partition(tbl, outPart));
         }
         getMSC().alter_partitions(addPartitionDesc.getDbName(), addPartitionDesc.getTableName(),
-            partsToAlter, new EnvironmentContext(), -1, null, -1);
+            partsToAlter, new EnvironmentContext(), null, -1);
 
         for ( org.apache.hadoop.hive.metastore.api.Partition outPart :
         getMSC().getPartitionsByNames(addPartitionDesc.getDbName(), addPartitionDesc.getTableName(),part_names)){
@@ -4594,7 +4588,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
       Table tbl = getTable(statsDesc.getDbName(), statsDesc.getTableName());
 
       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);
@@ -4613,11 +4606,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
       if (checkTransactional) {
         Table tbl = getTable(dbName, tableName);
         AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl);
-        if (tableSnapshot.getTxnId() > 0) {
-          retv = getMSC().getTableColumnStatistics(dbName, tableName, colNames,
-              tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
-              tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
-        }
+        retv = getMSC().getTableColumnStatistics(dbName, tableName, colNames,
+            tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
       } else {
         retv = getMSC().getTableColumnStatistics(dbName, tableName, colNames);
       }
@@ -4632,18 +4622,16 @@ private void constructOneLBLocationMap(FileStatus fSta,
       String dbName, String tableName, List<String> partNames, List<String> colNames,
       boolean checkTransactional)
       throws HiveException {
-    long txnId = -1;
     String writeIdList = null;
     try {
       if (checkTransactional) {
         Table tbl = getTable(dbName, tableName);
         AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl);
-        txnId = tableSnapshot != null ? tableSnapshot.getTxnId() : -1;
         writeIdList = tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null;
       }
 
-      return getMSC().getPartitionColumnStatistics(dbName, tableName, partNames, colNames,
-            txnId, writeIdList);
+      return getMSC().getPartitionColumnStatistics(
+          dbName, tableName, partNames, colNames, writeIdList);
     } catch (Exception e) {
       LOG.debug(StringUtils.stringifyException(e));
       throw new HiveException(e);
@@ -4652,17 +4640,14 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
   public AggrStats getAggrColStatsFor(String dbName, String tblName,
      List<String> colNames, List<String> partName, boolean checkTransactional) {
-    long txnId = -1;
     String writeIdList = null;
     try {
       if (checkTransactional) {
         Table tbl = getTable(dbName, tblName);
         AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl);
-        txnId = tableSnapshot != null ? tableSnapshot.getTxnId() : -1;
         writeIdList = tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null;
       }
-      return getMSC().getAggrColStatsFor(dbName, tblName, colNames, partName,
-          txnId, writeIdList);
+      return getMSC().getAggrColStatsFor(dbName, tblName, colNames, partName, writeIdList);
     } catch (Exception e) {
       LOG.debug(StringUtils.stringifyException(e));
       return new AggrStats(new ArrayList<ColumnStatisticsObj>(),0);

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index f7c9009..3240f2d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -159,14 +159,14 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
 
   @Override
   public void truncateTable(String dbName, String tableName,
-      List<String> partNames, long txnId, String validWriteIds, long writeId)
+      List<String> partNames, String validWriteIds, long writeId)
       throws TException {
     org.apache.hadoop.hive.metastore.api.Table table = getTempTable(dbName, tableName);
     if (table != null) {
       truncateTempTable(table);
       return;
     }
-    super.truncateTable(dbName, tableName, partNames, txnId, validWriteIds, writeId);
+    super.truncateTable(dbName, tableName, partNames, validWriteIds, writeId);
   }
 
   @Override
@@ -362,7 +362,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   @Override
   public void alter_table(String catName, String dbName, String tbl_name,
       org.apache.hadoop.hive.metastore.api.Table new_tbl,
-      EnvironmentContext envContext, long txnId, String validWriteIds)
+      EnvironmentContext envContext, String validWriteIds)
       throws InvalidOperationException, MetaException, TException {
     org.apache.hadoop.hive.metastore.api.Table old_tbl = getTempTable(dbName, tbl_name);
     if (old_tbl != null) {
@@ -370,8 +370,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       alterTempTable(dbName, tbl_name, old_tbl, new_tbl, null);
       return;
     }
-    super.alter_table(catName, dbName, tbl_name, new_tbl, envContext, txnId,
-        validWriteIds);
+    super.alter_table(catName, dbName, tbl_name, new_tbl, envContext, validWriteIds);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
index 6c922ee..b8d4375 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
@@ -475,7 +475,6 @@ public class StatsOptimizer extends Transform {
                     hive.getMSC().getTableColumnStatistics(
                       tbl.getDbName(), tbl.getTableName(),
                       Lists.newArrayList(colName),
-                      tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
                       tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
                 if (stats.isEmpty()) {
                   Logger.debug("No stats for " + tbl.getTableName() + " column " + colName);
@@ -541,7 +540,6 @@ public class StatsOptimizer extends Transform {
                   hive.getMSC().getTableColumnStatistics(
                     tbl.getDbName(), tbl.getTableName(),
                     Lists.newArrayList(colName),
-                      tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
                       tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
               if (stats.isEmpty()) {
                 Logger.debug("No stats for " + tbl.getTableName() + " column " + colName);
@@ -685,7 +683,6 @@ public class StatsOptimizer extends Transform {
               ColumnStatisticsData statData =
                   hive.getMSC().getTableColumnStatistics(
                     tbl.getDbName(), tbl.getTableName(), Lists.newArrayList(colName),
-                      tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
                       tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null)
                     .get(0).getStatsData();
               String name = colDesc.getTypeString().toUpperCase();
@@ -923,7 +920,6 @@ public class StatsOptimizer extends Transform {
 
       Map<String, List<ColumnStatisticsObj>> result = hive.getMSC().getPartitionColumnStatistics(
           tbl.getDbName(), tbl.getTableName(), partNames, Lists.newArrayList(colName),
-          tableSnapshot != null ? tableSnapshot.getTxnId() : -1,
           tableSnapshot != null ? tableSnapshot.getValidWriteIdList() : null);
       if (result.size() != parts.size()) {
         Logger.debug("Received " + result.size() + " stats for " + parts.size() + " partitions");

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java
index 39209b3..2e25ece 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/ColStatsProcessor.java
@@ -182,7 +182,6 @@ public class ColStatsProcessor implements IStatsProcessor {
     HiveTxnManager txnMgr = AcidUtils.isTransactionalTable(tbl)
         ? SessionState.get().getTxnMgr() : null;
     if (txnMgr != null) {
-      request.setTxnId(txnMgr.getCurrentTxnId());
       request.setValidWriteIdList(AcidUtils.getTableValidWriteIdList(conf,
           AcidUtils.getFullTableName(tbl.getDbName(), tbl.getTableName())).toString());
       request.setWriteId(txnMgr.getAllocatedTableWriteId(tbl.getDbName(), tbl.getTableName()));

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
index 2e4ce11..f34cb61 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
@@ -447,7 +447,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
     // TODO: we should probably skip updating if writeId is from an active txn
     boolean isTxnValid = (writeIdString == null) || ObjectStore.isCurrentStatsValidForTheQuery(
-        conf, db, tbl, params, statsWriteId , 0, writeIdString, false);
+        conf, db, tbl, params, statsWriteId , writeIdString, false);
     return getExistingStatsToUpdate(existingStats, params, isTxnValid);
   }
 
@@ -472,7 +472,7 @@ public class StatsUpdaterThread extends Thread implements MetaStoreThread {
     }
     // TODO: we should probably skip updating if writeId is from an active txn
     if (writeIdString != null && !ObjectStore.isCurrentStatsValidForTheQuery(
-        conf, db, tbl, params, statsWriteId, 0, writeIdString, false)) {
+        conf, db, tbl, params, statsWriteId, writeIdString, false)) {
       return allCols;
     }
     List<String> colsToUpdate = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 236bb7a..8c33f6a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -277,7 +277,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     List<ColumnStatisticsObj> stats;
     validWriteIds = msClient.getValidWriteIds("default." + tableName).toString();
     stats = msClient.getTableColumnStatistics(
-        "default", tableName, Lists.newArrayList("a"), -1, validWriteIds);
+        "default", tableName, Lists.newArrayList("a"), validWriteIds);
     return stats;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java
index 6c768c0..55131f3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUpdaterThread.java
@@ -153,13 +153,13 @@ public class TestStatsUpdaterThread {
     List<String> cols = Lists.newArrayList("s");
     String dbName = ss.getCurrentDatabase(), tblName = "simple_stats", fqName = dbName + "." + tblName;
     ValidWriteIdList initialWriteIds = msClient.getValidWriteIds(fqName);
-    verifyStatsUpToDate(tblName, cols, msClient, 0, initialWriteIds.toString(), true);
+    verifyStatsUpToDate(tblName, cols, msClient, initialWriteIds.toString(), true);
     assertFalse(su.runOneIteration());
     drainWorkQueue(su, 0);
 
     executeQuery("insert overwrite table simple_stats values ('test2')");
     ValidWriteIdList nextWriteIds = msClient.getValidWriteIds(fqName);
-    verifyStatsUpToDate(tblName, cols, msClient, 0, nextWriteIds.toString(), true);
+    verifyStatsUpToDate(tblName, cols, msClient, nextWriteIds.toString(), true);
     assertFalse(su.runOneIteration());
     drainWorkQueue(su, 0);
     String currentWriteIds = msClient.getValidWriteIds(fqName).toString();
@@ -171,17 +171,17 @@ public class TestStatsUpdaterThread {
     Table tbl = msClient.getTable(dbName, tblName);
     tbl.setWriteId(badWriteId);
     msClient.alter_table(
-        null, dbName, tblName, tbl, new EnvironmentContext(), -1, initialWriteIds.toString());
+        null, dbName, tblName, tbl, new EnvironmentContext(), initialWriteIds.toString());
 
     // Stats should not be valid.
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, false);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, false);
 
     // Analyze should not be able to set valid stats for a running txn.
     assertTrue(su.runOneIteration());
     drainWorkQueue(su);
 
     currentWriteIds = msClient.getValidWriteIds(fqName).toString();
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, false);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, false);
 
     msClient.abortTxns(Lists.newArrayList(badTxnId));
 
@@ -191,7 +191,7 @@ public class TestStatsUpdaterThread {
 
     // Stats will now be valid.
     currentWriteIds = msClient.getValidWriteIds(fqName).toString();
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, true);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, true);
 
     // Verify that incorrect stats from a valid write ID are also handled.
     badTxnId = msClient.openTxn("moo");
@@ -199,17 +199,17 @@ public class TestStatsUpdaterThread {
     tbl = msClient.getTable(dbName, tblName);
     tbl.setWriteId(badWriteId);
     StatsSetupConst.setBasicStatsState(tbl.getParameters(), StatsSetupConst.FALSE);
-    msClient.alter_table(null, dbName, tblName, tbl, new EnvironmentContext(), -1, initialWriteIds.toString());
+    msClient.alter_table(null, dbName, tblName, tbl, new EnvironmentContext(), initialWriteIds.toString());
 
     // Stats should not be valid.
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, false);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, false);
 
     // Analyze should not be able to set valid stats for a running txn.
     assertTrue(su.runOneIteration());
     drainWorkQueue(su);
 
     currentWriteIds = msClient.getValidWriteIds(fqName).toString();
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, false);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, false);
 
     msClient.commitTxn(badTxnId);
 
@@ -219,7 +219,7 @@ public class TestStatsUpdaterThread {
 
     // Stats will now be valid.
     currentWriteIds = msClient.getValidWriteIds(fqName).toString();
-    verifyStatsUpToDate(tblName, cols, msClient, 0, currentWriteIds, true);
+    verifyStatsUpToDate(tblName, cols, msClient, currentWriteIds, true);
 
     msClient.close();
   }
@@ -256,14 +256,14 @@ public class TestStatsUpdaterThread {
     String currentWriteIds = msClient.getValidWriteIds(fqName).toString();
     // To update write ID we need to specify the write ID list to validate concurrent writes.
     msClient.alter_partitions(dbName, tblName,
-        Lists.newArrayList(part1), null, -1, currentWriteIds, badWriteId);
+        Lists.newArrayList(part1), null, currentWriteIds, badWriteId);
     msClient.alter_partitions(dbName, tblName,
-        Lists.newArrayList(part2), null, -1, currentWriteIds, badWriteId);
+        Lists.newArrayList(part2), null, currentWriteIds, badWriteId);
 
     // We expect two partitions to be updated.
     Map<String, List<ColumnStatisticsObj>> stats = msClient.getPartitionColumnStatistics(
         dbName, tblName, Lists.newArrayList("p=1", "p=2", "p=3"),
-        Lists.newArrayList("s"), 0, currentWriteIds);
+        Lists.newArrayList("s"), currentWriteIds);
     assertEquals(1, stats.size());
 
     assertTrue(su.runOneIteration());
@@ -271,14 +271,14 @@ public class TestStatsUpdaterThread {
     // Analyze treats stats like data (new write ID), so stats still should not be valid.
     stats = msClient.getPartitionColumnStatistics(
         dbName, tblName, Lists.newArrayList("p=1", "p=2", "p=3"),
-        Lists.newArrayList("s"), 0, currentWriteIds);
+        Lists.newArrayList("s"), currentWriteIds);
     assertEquals(1, stats.size());
 
     // New reader.
     currentWriteIds = msClient.getValidWriteIds(fqName).toString();
     stats = msClient.getPartitionColumnStatistics(
         dbName, tblName, Lists.newArrayList("p=1", "p=2", "p=3"),
-        Lists.newArrayList("s"), 0, currentWriteIds);
+        Lists.newArrayList("s"), currentWriteIds);
     assertEquals(3, stats.size());
 
     msClient.close();
@@ -588,8 +588,8 @@ public class TestStatsUpdaterThread {
   }
 
   private void verifyStatsUpToDate(String tbl, List<String> cols, IMetaStoreClient msClient,
-      long txnId, String validWriteIds, boolean isUpToDate) throws Exception {
-    Table table = msClient.getTable(ss.getCurrentDatabase(), tbl, txnId, validWriteIds);
+      String validWriteIds, boolean isUpToDate) throws Exception {
+    Table table = msClient.getTable(ss.getCurrentCatalog(), ss.getCurrentDatabase(), tbl, validWriteIds);
     verifyStatsUpToDate(table.getParameters(), cols, isUpToDate);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
index ec42631..469a9a8 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddPartitionsRequest.java
@@ -44,8 +44,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField IF_NOT_EXISTS_FIELD_DESC = new org.apache.thrift.protocol.TField("ifNotExists", org.apache.thrift.protocol.TType.BOOL, (short)4);
   private static final org.apache.thrift.protocol.TField NEED_RESULT_FIELD_DESC = new org.apache.thrift.protocol.TField("needResult", org.apache.thrift.protocol.TType.BOOL, (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 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 {
@@ -59,7 +58,6 @@ import org.slf4j.LoggerFactory;
   private boolean ifNotExists; // required
   private boolean needResult; // optional
   private String catName; // optional
-  private long txnId; // optional
   private String validWriteIdList; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -70,8 +68,7 @@ import org.slf4j.LoggerFactory;
     IF_NOT_EXISTS((short)4, "ifNotExists"),
     NEED_RESULT((short)5, "needResult"),
     CAT_NAME((short)6, "catName"),
-    TXN_ID((short)7, "txnId"),
-    VALID_WRITE_ID_LIST((short)8, "validWriteIdList");
+    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -98,9 +95,7 @@ import org.slf4j.LoggerFactory;
           return NEED_RESULT;
         case 6: // CAT_NAME
           return CAT_NAME;
-        case 7: // TXN_ID
-          return TXN_ID;
-        case 8: // VALID_WRITE_ID_LIST
+        case 7: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
         default:
           return null;
@@ -144,9 +139,8 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __IFNOTEXISTS_ISSET_ID = 0;
   private static final int __NEEDRESULT_ISSET_ID = 1;
-  private static final int __TXNID_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.NEED_RESULT,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.NEED_RESULT,_Fields.CAT_NAME,_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);
@@ -163,8 +157,6 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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.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);
@@ -174,8 +166,6 @@ import org.slf4j.LoggerFactory;
   public AddPartitionsRequest() {
     this.needResult = true;
 
-    this.txnId = -1L;
-
   }
 
   public AddPartitionsRequest(
@@ -215,7 +205,6 @@ import org.slf4j.LoggerFactory;
     if (other.isSetCatName()) {
       this.catName = other.catName;
     }
-    this.txnId = other.txnId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
@@ -235,8 +224,6 @@ import org.slf4j.LoggerFactory;
     this.needResult = true;
 
     this.catName = null;
-    this.txnId = -1L;
-
     this.validWriteIdList = null;
   }
 
@@ -391,28 +378,6 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  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 String getValidWriteIdList() {
     return this.validWriteIdList;
   }
@@ -486,14 +451,6 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case TXN_ID:
-      if (value == null) {
-        unsetTxnId();
-      } else {
-        setTxnId((Long)value);
-      }
-      break;
-
     case VALID_WRITE_ID_LIST:
       if (value == null) {
         unsetValidWriteIdList();
@@ -525,9 +482,6 @@ import org.slf4j.LoggerFactory;
     case CAT_NAME:
       return getCatName();
 
-    case TXN_ID:
-      return getTxnId();
-
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
@@ -554,8 +508,6 @@ import org.slf4j.LoggerFactory;
       return isSetNeedResult();
     case CAT_NAME:
       return isSetCatName();
-    case TXN_ID:
-      return isSetTxnId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
     }
@@ -629,15 +581,6 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
-    boolean this_present_txnId = true && this.isSetTxnId();
-    boolean that_present_txnId = true && that.isSetTxnId();
-    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_validWriteIdList = true && this.isSetValidWriteIdList();
     boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
     if (this_present_validWriteIdList || that_present_validWriteIdList) {
@@ -684,11 +627,6 @@ import org.slf4j.LoggerFactory;
     if (present_catName)
       list.add(catName);
 
-    boolean present_txnId = true && (isSetTxnId());
-    list.add(present_txnId);
-    if (present_txnId)
-      list.add(txnId);
-
     boolean present_validWriteIdList = true && (isSetValidWriteIdList());
     list.add(present_validWriteIdList);
     if (present_validWriteIdList)
@@ -765,16 +703,6 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    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(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
@@ -848,12 +776,6 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
-    if (isSetTxnId()) {
-      if (!first) sb.append(", ");
-      sb.append("txnId:");
-      sb.append(this.txnId);
-      first = false;
-    }
     if (isSetValidWriteIdList()) {
       if (!first) sb.append(", ");
       sb.append("validWriteIdList:");
@@ -984,15 +906,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // 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 8: // VALID_WRITE_ID_LIST
+          case 7: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -1050,11 +964,6 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
-        oprot.writeI64(struct.txnId);
-        oprot.writeFieldEnd();
-      }
       if (struct.validWriteIdList != null) {
         if (struct.isSetValidWriteIdList()) {
           oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
@@ -1096,22 +1005,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCatName()) {
         optionals.set(1);
       }
-      if (struct.isSetTxnId()) {
-        optionals.set(2);
-      }
       if (struct.isSetValidWriteIdList()) {
-        optionals.set(3);
+        optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 4);
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetNeedResult()) {
         oprot.writeBool(struct.needResult);
       }
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeI64(struct.txnId);
-      }
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
@@ -1138,7 +1041,7 @@ import org.slf4j.LoggerFactory;
       struct.setPartsIsSet(true);
       struct.ifNotExists = iprot.readBool();
       struct.setIfNotExistsIsSet(true);
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.needResult = iprot.readBool();
         struct.setNeedResultIsSet(true);
@@ -1148,10 +1051,6 @@ import org.slf4j.LoggerFactory;
         struct.setCatNameIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.txnId = iprot.readI64();
-        struct.setTxnIdIsSet(true);
-      }
-      if (incoming.get(3)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
index 45832a4..4d4595a 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
@@ -43,9 +43,8 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)3);
   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)4);
   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)5);
-  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)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)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 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 {
@@ -58,7 +57,6 @@ import org.slf4j.LoggerFactory;
   private String tableName; // required
   private List<Partition> partitions; // required
   private EnvironmentContext environmentContext; // optional
-  private long txnId; // optional
   private long writeId; // optional
   private String validWriteIdList; // optional
 
@@ -69,9 +67,8 @@ import org.slf4j.LoggerFactory;
     TABLE_NAME((short)3, "tableName"),
     PARTITIONS((short)4, "partitions"),
     ENVIRONMENT_CONTEXT((short)5, "environmentContext"),
-    TXN_ID((short)6, "txnId"),
-    WRITE_ID((short)7, "writeId"),
-    VALID_WRITE_ID_LIST((short)8, "validWriteIdList");
+    WRITE_ID((short)6, "writeId"),
+    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -96,11 +93,9 @@ import org.slf4j.LoggerFactory;
           return PARTITIONS;
         case 5: // ENVIRONMENT_CONTEXT
           return ENVIRONMENT_CONTEXT;
-        case 6: // TXN_ID
-          return TXN_ID;
-        case 7: // WRITE_ID
+        case 6: // WRITE_ID
           return WRITE_ID;
-        case 8: // VALID_WRITE_ID_LIST
+        case 7: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
         default:
           return null;
@@ -142,10 +137,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 static final int __WRITEID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_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);
@@ -160,8 +154,6 @@ import org.slf4j.LoggerFactory;
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Partition.class))));
     tmpMap.put(_Fields.ENVIRONMENT_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("environmentContext", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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, 
@@ -171,8 +163,6 @@ import org.slf4j.LoggerFactory;
   }
 
   public AlterPartitionsRequest() {
-    this.txnId = -1L;
-
     this.writeId = -1L;
 
   }
@@ -212,7 +202,6 @@ import org.slf4j.LoggerFactory;
     if (other.isSetEnvironmentContext()) {
       this.environmentContext = new EnvironmentContext(other.environmentContext);
     }
-    this.txnId = other.txnId;
     this.writeId = other.writeId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
@@ -230,8 +219,6 @@ import org.slf4j.LoggerFactory;
     this.tableName = null;
     this.partitions = null;
     this.environmentContext = null;
-    this.txnId = -1L;
-
     this.writeId = -1L;
 
     this.validWriteIdList = null;
@@ -367,28 +354,6 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  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;
   }
@@ -476,14 +441,6 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case TXN_ID:
-      if (value == null) {
-        unsetTxnId();
-      } else {
-        setTxnId((Long)value);
-      }
-      break;
-
     case WRITE_ID:
       if (value == null) {
         unsetWriteId();
@@ -520,9 +477,6 @@ import org.slf4j.LoggerFactory;
     case ENVIRONMENT_CONTEXT:
       return getEnvironmentContext();
 
-    case TXN_ID:
-      return getTxnId();
-
     case WRITE_ID:
       return getWriteId();
 
@@ -550,8 +504,6 @@ import org.slf4j.LoggerFactory;
       return isSetPartitions();
     case ENVIRONMENT_CONTEXT:
       return isSetEnvironmentContext();
-    case TXN_ID:
-      return isSetTxnId();
     case WRITE_ID:
       return isSetWriteId();
     case VALID_WRITE_ID_LIST:
@@ -618,15 +570,6 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
-    boolean this_present_txnId = true && this.isSetTxnId();
-    boolean that_present_txnId = true && that.isSetTxnId();
-    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 && this.isSetWriteId();
     boolean that_present_writeId = true && that.isSetWriteId();
     if (this_present_writeId || that_present_writeId) {
@@ -677,11 +620,6 @@ import org.slf4j.LoggerFactory;
     if (present_environmentContext)
       list.add(environmentContext);
 
-    boolean present_txnId = true && (isSetTxnId());
-    list.add(present_txnId);
-    if (present_txnId)
-      list.add(txnId);
-
     boolean present_writeId = true && (isSetWriteId());
     list.add(present_writeId);
     if (present_writeId)
@@ -753,16 +691,6 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    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;
@@ -846,12 +774,6 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
-    if (isSetTxnId()) {
-      if (!first) sb.append(", ");
-      sb.append("txnId:");
-      sb.append(this.txnId);
-      first = false;
-    }
     if (isSetWriteId()) {
       if (!first) sb.append(", ");
       sb.append("writeId:");
@@ -980,15 +902,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // 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 7: // WRITE_ID
+          case 6: // WRITE_ID
             if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
               struct.writeId = iprot.readI64();
               struct.setWriteIdIsSet(true);
@@ -996,7 +910,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // VALID_WRITE_ID_LIST
+          case 7: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -1053,11 +967,6 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
-        oprot.writeI64(struct.txnId);
-        oprot.writeFieldEnd();
-      }
       if (struct.isSetWriteId()) {
         oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
         oprot.writeI64(struct.writeId);
@@ -1103,25 +1012,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetEnvironmentContext()) {
         optionals.set(1);
       }
-      if (struct.isSetTxnId()) {
-        optionals.set(2);
-      }
       if (struct.isSetWriteId()) {
-        optionals.set(3);
+        optionals.set(2);
       }
       if (struct.isSetValidWriteIdList()) {
-        optionals.set(4);
+        optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 5);
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
       if (struct.isSetEnvironmentContext()) {
         struct.environmentContext.write(oprot);
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeI64(struct.txnId);
-      }
       if (struct.isSetWriteId()) {
         oprot.writeI64(struct.writeId);
       }
@@ -1149,7 +1052,7 @@ import org.slf4j.LoggerFactory;
         }
       }
       struct.setPartitionsIsSet(true);
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         struct.catName = iprot.readString();
         struct.setCatNameIsSet(true);
@@ -1160,14 +1063,10 @@ import org.slf4j.LoggerFactory;
         struct.setEnvironmentContextIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.txnId = iprot.readI64();
-        struct.setTxnIdIsSet(true);
-      }
-      if (incoming.get(3)) {
         struct.writeId = iprot.readI64();
         struct.setWriteIdIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(3)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
index 6a1efd8..df295c7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterTableRequest.java
@@ -43,9 +43,8 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", 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.STRUCT, (short)4);
   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)5);
-  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)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)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 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 {
@@ -58,7 +57,6 @@ import org.slf4j.LoggerFactory;
   private String tableName; // required
   private Table table; // required
   private EnvironmentContext environmentContext; // optional
-  private long txnId; // optional
   private long writeId; // optional
   private String validWriteIdList; // optional
 
@@ -69,9 +67,8 @@ import org.slf4j.LoggerFactory;
     TABLE_NAME((short)3, "tableName"),
     TABLE((short)4, "table"),
     ENVIRONMENT_CONTEXT((short)5, "environmentContext"),
-    TXN_ID((short)6, "txnId"),
-    WRITE_ID((short)7, "writeId"),
-    VALID_WRITE_ID_LIST((short)8, "validWriteIdList");
+    WRITE_ID((short)6, "writeId"),
+    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -96,11 +93,9 @@ import org.slf4j.LoggerFactory;
           return TABLE;
         case 5: // ENVIRONMENT_CONTEXT
           return ENVIRONMENT_CONTEXT;
-        case 6: // TXN_ID
-          return TXN_ID;
-        case 7: // WRITE_ID
+        case 6: // WRITE_ID
           return WRITE_ID;
-        case 8: // VALID_WRITE_ID_LIST
+        case 7: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
         default:
           return null;
@@ -142,10 +137,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 static final int __WRITEID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.ENVIRONMENT_CONTEXT,_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);
@@ -159,8 +153,6 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Table.class)));
     tmpMap.put(_Fields.ENVIRONMENT_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("environmentContext", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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, 
@@ -170,8 +162,6 @@ import org.slf4j.LoggerFactory;
   }
 
   public AlterTableRequest() {
-    this.txnId = -1L;
-
     this.writeId = -1L;
 
   }
@@ -207,7 +197,6 @@ import org.slf4j.LoggerFactory;
     if (other.isSetEnvironmentContext()) {
       this.environmentContext = new EnvironmentContext(other.environmentContext);
     }
-    this.txnId = other.txnId;
     this.writeId = other.writeId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
@@ -225,8 +214,6 @@ import org.slf4j.LoggerFactory;
     this.tableName = null;
     this.table = null;
     this.environmentContext = null;
-    this.txnId = -1L;
-
     this.writeId = -1L;
 
     this.validWriteIdList = null;
@@ -347,28 +334,6 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  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;
   }
@@ -456,14 +421,6 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case TXN_ID:
-      if (value == null) {
-        unsetTxnId();
-      } else {
-        setTxnId((Long)value);
-      }
-      break;
-
     case WRITE_ID:
       if (value == null) {
         unsetWriteId();
@@ -500,9 +457,6 @@ import org.slf4j.LoggerFactory;
     case ENVIRONMENT_CONTEXT:
       return getEnvironmentContext();
 
-    case TXN_ID:
-      return getTxnId();
-
     case WRITE_ID:
       return getWriteId();
 
@@ -530,8 +484,6 @@ import org.slf4j.LoggerFactory;
       return isSetTable();
     case ENVIRONMENT_CONTEXT:
       return isSetEnvironmentContext();
-    case TXN_ID:
-      return isSetTxnId();
     case WRITE_ID:
       return isSetWriteId();
     case VALID_WRITE_ID_LIST:
@@ -598,15 +550,6 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
-    boolean this_present_txnId = true && this.isSetTxnId();
-    boolean that_present_txnId = true && that.isSetTxnId();
-    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 && this.isSetWriteId();
     boolean that_present_writeId = true && that.isSetWriteId();
     if (this_present_writeId || that_present_writeId) {
@@ -657,11 +600,6 @@ import org.slf4j.LoggerFactory;
     if (present_environmentContext)
       list.add(environmentContext);
 
-    boolean present_txnId = true && (isSetTxnId());
-    list.add(present_txnId);
-    if (present_txnId)
-      list.add(txnId);
-
     boolean present_writeId = true && (isSetWriteId());
     list.add(present_writeId);
     if (present_writeId)
@@ -733,16 +671,6 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    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;
@@ -826,12 +754,6 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
-    if (isSetTxnId()) {
-      if (!first) sb.append(", ");
-      sb.append("txnId:");
-      sb.append(this.txnId);
-      first = false;
-    }
     if (isSetWriteId()) {
       if (!first) sb.append(", ");
       sb.append("writeId:");
@@ -953,15 +875,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 6: // 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 7: // WRITE_ID
+          case 6: // WRITE_ID
             if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
               struct.writeId = iprot.readI64();
               struct.setWriteIdIsSet(true);
@@ -969,7 +883,7 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 8: // VALID_WRITE_ID_LIST
+          case 7: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
               struct.setValidWriteIdListIsSet(true);
@@ -1019,11 +933,6 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
-        oprot.writeI64(struct.txnId);
-        oprot.writeFieldEnd();
-      }
       if (struct.isSetWriteId()) {
         oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
         oprot.writeI64(struct.writeId);
@@ -1063,25 +972,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetEnvironmentContext()) {
         optionals.set(1);
       }
-      if (struct.isSetTxnId()) {
-        optionals.set(2);
-      }
       if (struct.isSetWriteId()) {
-        optionals.set(3);
+        optionals.set(2);
       }
       if (struct.isSetValidWriteIdList()) {
-        optionals.set(4);
+        optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 5);
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
       if (struct.isSetEnvironmentContext()) {
         struct.environmentContext.write(oprot);
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeI64(struct.txnId);
-      }
       if (struct.isSetWriteId()) {
         oprot.writeI64(struct.writeId);
       }
@@ -1100,7 +1003,7 @@ import org.slf4j.LoggerFactory;
       struct.table = new Table();
       struct.table.read(iprot);
       struct.setTableIsSet(true);
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         struct.catName = iprot.readString();
         struct.setCatNameIsSet(true);
@@ -1111,14 +1014,10 @@ import org.slf4j.LoggerFactory;
         struct.setEnvironmentContextIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.txnId = iprot.readI64();
-        struct.setTxnIdIsSet(true);
-      }
-      if (incoming.get(3)) {
         struct.writeId = iprot.readI64();
         struct.setWriteIdIsSet(true);
       }
-      if (incoming.get(4)) {
+      if (incoming.get(3)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
index 821049e..2804952 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTableRequest.java
@@ -42,7 +42,6 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField CAPABILITIES_FIELD_DESC = new org.apache.thrift.protocol.TField("capabilities", org.apache.thrift.protocol.TType.STRUCT, (short)3);
   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)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 Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
@@ -55,7 +54,6 @@ import org.slf4j.LoggerFactory;
   private String tblName; // required
   private ClientCapabilities capabilities; // optional
   private String catName; // optional
-  private long txnId; // optional
   private String validWriteIdList; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -64,7 +62,6 @@ import org.slf4j.LoggerFactory;
     TBL_NAME((short)2, "tblName"),
     CAPABILITIES((short)3, "capabilities"),
     CAT_NAME((short)4, "catName"),
-    TXN_ID((short)5, "txnId"),
     VALID_WRITE_ID_LIST((short)6, "validWriteIdList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -88,8 +85,6 @@ import org.slf4j.LoggerFactory;
           return CAPABILITIES;
         case 4: // CAT_NAME
           return CAT_NAME;
-        case 5: // TXN_ID
-          return TXN_ID;
         case 6: // VALID_WRITE_ID_LIST
           return VALID_WRITE_ID_LIST;
         default:
@@ -132,9 +127,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.CAPABILITIES,_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = {_Fields.CAPABILITIES,_Fields.CAT_NAME,_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);
@@ -146,8 +139,6 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClientCapabilities.class)));
     tmpMap.put(_Fields.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         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.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);
@@ -155,8 +146,6 @@ import org.slf4j.LoggerFactory;
   }
 
   public GetTableRequest() {
-    this.txnId = -1L;
-
   }
 
   public GetTableRequest(
@@ -172,7 +161,6 @@ import org.slf4j.LoggerFactory;
    * Performs a deep copy on <i>other</i>.
    */
   public GetTableRequest(GetTableRequest other) {
-    __isset_bitfield = other.__isset_bitfield;
     if (other.isSetDbName()) {
       this.dbName = other.dbName;
     }
@@ -185,7 +173,6 @@ import org.slf4j.LoggerFactory;
     if (other.isSetCatName()) {
       this.catName = other.catName;
     }
-    this.txnId = other.txnId;
     if (other.isSetValidWriteIdList()) {
       this.validWriteIdList = other.validWriteIdList;
     }
@@ -201,8 +188,6 @@ import org.slf4j.LoggerFactory;
     this.tblName = null;
     this.capabilities = null;
     this.catName = null;
-    this.txnId = -1L;
-
     this.validWriteIdList = null;
   }
 
@@ -298,28 +283,6 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  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 String getValidWriteIdList() {
     return this.validWriteIdList;
   }
@@ -377,14 +340,6 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case TXN_ID:
-      if (value == null) {
-        unsetTxnId();
-      } else {
-        setTxnId((Long)value);
-      }
-      break;
-
     case VALID_WRITE_ID_LIST:
       if (value == null) {
         unsetValidWriteIdList();
@@ -410,9 +365,6 @@ import org.slf4j.LoggerFactory;
     case CAT_NAME:
       return getCatName();
 
-    case TXN_ID:
-      return getTxnId();
-
     case VALID_WRITE_ID_LIST:
       return getValidWriteIdList();
 
@@ -435,8 +387,6 @@ import org.slf4j.LoggerFactory;
       return isSetCapabilities();
     case CAT_NAME:
       return isSetCatName();
-    case TXN_ID:
-      return isSetTxnId();
     case VALID_WRITE_ID_LIST:
       return isSetValidWriteIdList();
     }
@@ -492,15 +442,6 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
-    boolean this_present_txnId = true && this.isSetTxnId();
-    boolean that_present_txnId = true && that.isSetTxnId();
-    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_validWriteIdList = true && this.isSetValidWriteIdList();
     boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
     if (this_present_validWriteIdList || that_present_validWriteIdList) {
@@ -537,11 +478,6 @@ import org.slf4j.LoggerFactory;
     if (present_catName)
       list.add(catName);
 
-    boolean present_txnId = true && (isSetTxnId());
-    list.add(present_txnId);
-    if (present_txnId)
-      list.add(txnId);
-
     boolean present_validWriteIdList = true && (isSetValidWriteIdList());
     list.add(present_validWriteIdList);
     if (present_validWriteIdList)
@@ -598,16 +534,6 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    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(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
     if (lastComparison != 0) {
       return lastComparison;
@@ -673,12 +599,6 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
-    if (isSetTxnId()) {
-      if (!first) sb.append(", ");
-      sb.append("txnId:");
-      sb.append(this.txnId);
-      first = false;
-    }
     if (isSetValidWriteIdList()) {
       if (!first) sb.append(", ");
       sb.append("validWriteIdList:");
@@ -719,8 +639,6 @@ import org.slf4j.LoggerFactory;
 
   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);
@@ -778,14 +696,6 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 5: // 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 6: // VALID_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.validWriteIdList = iprot.readString();
@@ -831,11 +741,6 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
-        oprot.writeI64(struct.txnId);
-        oprot.writeFieldEnd();
-      }
       if (struct.validWriteIdList != null) {
         if (struct.isSetValidWriteIdList()) {
           oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
@@ -869,22 +774,16 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCatName()) {
         optionals.set(1);
       }
-      if (struct.isSetTxnId()) {
-        optionals.set(2);
-      }
       if (struct.isSetValidWriteIdList()) {
-        optionals.set(3);
+        optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 4);
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetCapabilities()) {
         struct.capabilities.write(oprot);
       }
       if (struct.isSetCatName()) {
         oprot.writeString(struct.catName);
       }
-      if (struct.isSetTxnId()) {
-        oprot.writeI64(struct.txnId);
-      }
       if (struct.isSetValidWriteIdList()) {
         oprot.writeString(struct.validWriteIdList);
       }
@@ -897,7 +796,7 @@ import org.slf4j.LoggerFactory;
       struct.setDbNameIsSet(true);
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.capabilities = new ClientCapabilities();
         struct.capabilities.read(iprot);
@@ -908,10 +807,6 @@ import org.slf4j.LoggerFactory;
         struct.setCatNameIsSet(true);
       }
       if (incoming.get(2)) {
-        struct.txnId = iprot.readI64();
-        struct.setTxnIdIsSet(true);
-      }
-      if (incoming.get(3)) {
         struct.validWriteIdList = iprot.readString();
         struct.setValidWriteIdListIsSet(true);
       }