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:07 UTC

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

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 70edb96..91c86a7 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -420,7 +420,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public void alter_table(String catName, String dbName, String tbl_name, Table new_tbl,
-      EnvironmentContext envContext, long txnId, String validWriteIds)
+      EnvironmentContext envContext, String validWriteIds)
           throws InvalidOperationException, MetaException, TException {
     HiveMetaHook hook = getHook(new_tbl);
     if (hook != null) {
@@ -428,7 +428,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
     AlterTableRequest req = new AlterTableRequest(dbName, tbl_name, new_tbl);
     req.setCatName(catName);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(validWriteIds);
     req.setEnvironmentContext(envContext);
     client.alter_table_req(req);
@@ -438,15 +437,14 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public void renamePartition(final String dbname, final String tableName, final List<String> part_vals,
                               final Partition newPart) throws TException {
-    renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart, -1, null);
+    renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, newPart, null);
   }
 
   @Override
   public void renamePartition(String catName, String dbname, String tableName, List<String> part_vals,
-                              Partition newPart, long txnId, String validWriteIds) throws TException {
+                              Partition newPart, String validWriteIds) throws TException {
     RenamePartitionRequest req = new RenamePartitionRequest(dbname, tableName, part_vals, newPart);
     req.setCatName(catName);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(validWriteIds);
     client.rename_partition_req(req);
   }
@@ -816,35 +814,34 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String dbName, String tableName, List<String> partNames, List<String> colNames,
-      long txnId, String validWriteIdList)
+      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     return getPartitionColumnStatistics(getDefaultCatalog(conf), dbName, tableName,
-        partNames, colNames, txnId, validWriteIdList);
+        partNames, colNames, validWriteIdList);
   }
 
   @Override
   public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String catName, String dbName, String tableName, List<String> partNames,
-      List<String> colNames, long txnId, String validWriteIdList)
+      List<String> colNames, String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, tableName, colNames,
         partNames);
     rqst.setCatName(catName);
-    rqst.setTxnId(txnId);
     rqst.setValidWriteIdList(validWriteIdList);
     return client.get_partitions_statistics_req(rqst).getPartStats();
   }
 
   @Override
   public AggrStats getAggrColStatsFor(String dbName, String tblName, List<String> colNames,
-      List<String> partNames, long txnId, String writeIdList)
+      List<String> partNames, String writeIdList)
       throws NoSuchObjectException, MetaException, TException {
     return getAggrColStatsFor(getDefaultCatalog(conf), dbName, tblName, colNames,
-        partNames, txnId, writeIdList);  }
+        partNames, writeIdList);  }
 
   @Override
   public AggrStats getAggrColStatsFor(String catName, String dbName, String tblName, List<String> colNames,
-      List<String> partNames, long txnId, String writeIdList)
+      List<String> partNames, String writeIdList)
       throws NoSuchObjectException, MetaException, TException {
     if (colNames.isEmpty() || partNames.isEmpty()) {
       LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side.");
@@ -852,7 +849,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
     PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partNames);
     req.setCatName(catName);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(writeIdList);
     return client.get_aggr_stats_for(req);
   }
@@ -1462,29 +1458,28 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public void truncateTable(String dbName, String tableName, List<String> partNames,
-      long txnId, String validWriteIds, long writeId) throws TException {
+      String validWriteIds, long writeId) throws TException {
     truncateTableInternal(getDefaultCatalog(conf),
-        dbName, tableName, partNames, txnId, validWriteIds, writeId);
+        dbName, tableName, partNames, validWriteIds, writeId);
   }
 
   @Override
   public void truncateTable(String dbName, String tableName, List<String> partNames) throws TException {
-    truncateTableInternal(getDefaultCatalog(conf), dbName, tableName, partNames, -1, null, -1);
+    truncateTableInternal(getDefaultCatalog(conf), dbName, tableName, partNames, null, -1);
   }
 
   @Override
   public void truncateTable(String catName, String dbName, String tableName, List<String> partNames)
       throws TException {
-    truncateTableInternal(catName, dbName, tableName, partNames, -1, null, -1);
+    truncateTableInternal(catName, dbName, tableName, partNames, null, -1);
   }
 
   private void truncateTableInternal(String catName, String dbName, String tableName,
-      List<String> partNames, long txnId, String validWriteIds, long writeId)
+      List<String> partNames, String validWriteIds, long writeId)
           throws MetaException, TException {
     TruncateTableRequest req = new TruncateTableRequest(
         prependCatalogToDbName(catName, dbName, conf), tableName);
     req.setPartNames(partNames);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(validWriteIds);
     req.setWriteId(writeId);
     client.truncate_table_req(req);
@@ -1769,14 +1764,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public Table getTable(String dbname, String name,
-                 long txnId, String validWriteIdList)
-      throws MetaException, TException, NoSuchObjectException{
-    return getTable(getDefaultCatalog(conf), dbname, name,
-        txnId, validWriteIdList);
-  };
-
-  @Override
   public Table getTable(String catName, String dbName, String tableName) throws TException {
     GetTableRequest req = new GetTableRequest(dbName, tableName);
     req.setCatName(catName);
@@ -1787,11 +1774,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public Table getTable(String catName, String dbName, String tableName,
-    long txnId, String validWriteIdList) throws TException {
+    String validWriteIdList) throws TException {
     GetTableRequest req = new GetTableRequest(dbName, tableName);
     req.setCatName(catName);
     req.setCapabilities(version);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(validWriteIdList);
     Table t = client.get_table_req(req).getTable();
     return deepCopy(filterHook.filterTable(t));
@@ -2027,12 +2013,11 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public void alter_partition(String dbName, String tblName, Partition newPart,
-      EnvironmentContext environmentContext, long txnId, String writeIdList)
+      EnvironmentContext environmentContext, String writeIdList)
       throws InvalidOperationException, MetaException, TException {
     AlterPartitionsRequest req = new AlterPartitionsRequest(
         dbName, tblName, Lists.newArrayList(newPart));
     req.setEnvironmentContext(environmentContext);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(writeIdList);
     client.alter_partitions_req(req);
   }
@@ -2042,23 +2027,23 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public void alter_partitions(String dbName, String tblName, List<Partition> newParts)
       throws TException {
     alter_partitions(
-        getDefaultCatalog(conf), dbName, tblName, newParts, new EnvironmentContext(), -1, null, -1);
+        getDefaultCatalog(conf), dbName, tblName, newParts, new EnvironmentContext(), null, -1);
   }
 
   @Override
   public void alter_partitions(String dbName, String tblName, List<Partition> newParts,
                                EnvironmentContext environmentContext) throws TException {
     alter_partitions(
-        getDefaultCatalog(conf), dbName, tblName, newParts, environmentContext, -1, null, -1);
+        getDefaultCatalog(conf), dbName, tblName, newParts, environmentContext, null, -1);
   }
 
   @Override
   public void alter_partitions(String dbName, String tblName, List<Partition> newParts,
                                EnvironmentContext environmentContext,
-                               long txnId, String writeIdList, long writeId)
+                               String writeIdList, long writeId)
       throws InvalidOperationException, MetaException, TException {
     alter_partitions(getDefaultCatalog(conf),
-        dbName, tblName, newParts, environmentContext, txnId, writeIdList, writeId);
+        dbName, tblName, newParts, environmentContext, writeIdList, writeId);
 
   }
 
@@ -2066,14 +2051,13 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public void alter_partitions(String catName, String dbName, String tblName,
                                List<Partition> newParts,
                                EnvironmentContext environmentContext,
-                               long txnId, String writeIdList, long writeId) throws TException {
+                               String writeIdList, long writeId) throws TException {
     AlterPartitionsRequest req = new AlterPartitionsRequest();
     req.setCatName(catName);
     req.setDbName(dbName);
     req.setTableName(tblName);
     req.setPartitions(newParts);
     req.setEnvironmentContext(environmentContext);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(writeIdList);
     req.setWriteId(writeId);
     client.alter_partitions_req(req);
@@ -2218,21 +2202,18 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName,
                                                             List<String> colNames,
-                                                            long txnId,
                                                             String validWriteIdList) throws TException {
     return getTableColumnStatistics(getDefaultCatalog(conf), dbName, tableName, colNames,
-        txnId, validWriteIdList);
+        validWriteIdList);
   }
 
   @Override
   public List<ColumnStatisticsObj> getTableColumnStatistics(String catName, String dbName,
                                                             String tableName,
                                                             List<String> colNames,
-                                                            long txnId,
                                                             String validWriteIdList) throws TException {
     TableStatsRequest rqst = new TableStatsRequest(dbName, tableName, colNames);
     rqst.setCatName(catName);
-    rqst.setTxnId(txnId);
     rqst.setValidWriteIdList(validWriteIdList);
     return client.get_table_statistics_req(rqst).getTableStats();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
index 3a65f77..cee3572 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
@@ -91,7 +91,7 @@ public interface IHMSHandler extends ThriftHiveMetastore.Iface, Configurable {
       throws MetaException, NoSuchObjectException;
 
   Table get_table_core(final String catName, final String dbname,
-                       final String name, final long txnId,
+                       final String name,
                        final String writeIdList)
       throws MetaException, NoSuchObjectException;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 70be8d8..91405b9 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -624,7 +624,7 @@ public interface IMetaStoreClient {
   void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException;
 
   void truncateTable(String dbName, String tableName, List<String> partNames,
-      long txnId, String validWriteIds, long writeId) throws TException;
+      String validWriteIds, long writeId) throws TException;
 
   /**
    * Truncate the table/partitions in the DEFAULT database.
@@ -717,10 +717,6 @@ public interface IMetaStoreClient {
   Table getTable(String dbName, String tableName) throws MetaException,
       TException, NoSuchObjectException;
 
-  Table getTable(String dbName, String tableName,
-                 long txnId, String validWriteIdList)
-      throws MetaException, TException, NoSuchObjectException;
-
   /**
    * Get a table object.
    * @param catName catalog the table is in.
@@ -733,7 +729,7 @@ public interface IMetaStoreClient {
   Table getTable(String catName, String dbName, String tableName) throws MetaException, TException;
 
   Table getTable(String catName, String dbName, String tableName,
-                        long txnId, String validWriteIdList) throws TException;
+                        String validWriteIdList) throws TException;
   /**
    * Get tables as objects (rather than just fetching their names).  This is more expensive and
    * should only be used if you actually need all the information about the tables.
@@ -1650,7 +1646,7 @@ public interface IMetaStoreClient {
       TException;
 
   void alter_table(String catName, String databaseName, String tblName, Table table,
-      EnvironmentContext environmentContext, long txnId, String validWriteIdList)
+      EnvironmentContext environmentContext, String validWriteIdList)
           throws InvalidOperationException, MetaException, TException;
   /**
    * Create a new database.
@@ -2084,7 +2080,7 @@ public interface IMetaStoreClient {
 
 
   void alter_partition(String dbName, String tblName, Partition newPart,
-      EnvironmentContext environmentContext, long txnId, String writeIdList)
+      EnvironmentContext environmentContext, String writeIdList)
       throws InvalidOperationException, MetaException, TException;
 
   /**
@@ -2151,7 +2147,7 @@ public interface IMetaStoreClient {
 
   void alter_partitions(String dbName, String tblName, List<Partition> newParts,
                         EnvironmentContext environmentContext,
-                        long txnId, String writeIdList, long writeId)
+                        String writeIdList, long writeId)
       throws InvalidOperationException, MetaException, TException;
 
   /**
@@ -2174,7 +2170,7 @@ public interface IMetaStoreClient {
   default void alter_partitions(String catName, String dbName, String tblName,
                                 List<Partition> newParts)
       throws InvalidOperationException, MetaException, TException {
-    alter_partitions(catName, dbName, tblName, newParts, new EnvironmentContext(), -1, null, -1);
+    alter_partitions(catName, dbName, tblName, newParts, new EnvironmentContext(),  null, -1);
   }
 
   /**
@@ -2196,7 +2192,7 @@ public interface IMetaStoreClient {
    */
   void alter_partitions(String catName, String dbName, String tblName, List<Partition> newParts,
                         EnvironmentContext environmentContext,
-                        long txnId, String writeIdList, long writeId)
+                        String writeIdList, long writeId)
       throws InvalidOperationException, MetaException, TException;
 
   /**
@@ -2241,7 +2237,7 @@ public interface IMetaStoreClient {
    *          if error in communicating with metastore server
    */
   void renamePartition(String catName, String dbname, String tableName, List<String> part_vals,
-                       Partition newPart, long txnId, String validWriteIds)
+                       Partition newPart, String validWriteIds)
       throws InvalidOperationException, MetaException, TException;
 
   /**
@@ -2380,7 +2376,6 @@ public interface IMetaStoreClient {
 
   List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName,
                                                      List<String> colNames,
-                                                     long txnId,
                                                      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException;
 
@@ -2403,7 +2398,6 @@ public interface IMetaStoreClient {
 
   List<ColumnStatisticsObj> getTableColumnStatistics(String catName, String dbName, String tableName,
                                                      List<String> colNames,
-                                                     long txnId,
                                                      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException;
   /**
@@ -2424,7 +2418,7 @@ public interface IMetaStoreClient {
 
   Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(String dbName,
       String tableName,  List<String> partNames, List<String> colNames,
-      long txnId, String validWriteIdList)
+      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException;
 
   /**
@@ -2447,7 +2441,7 @@ public interface IMetaStoreClient {
   Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String catName, String dbName, String tableName,
       List<String> partNames, List<String> colNames,
-      long txnId, String validWriteIdList)
+      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException;
   /**
    * Delete partition level column statistics given dbName, tableName, partName and colName, or
@@ -3300,7 +3294,7 @@ public interface IMetaStoreClient {
 
   AggrStats getAggrColStatsFor(String dbName, String tblName,
       List<String> colNames, List<String> partName,
-      long txnId, String writeIdList)  throws NoSuchObjectException, MetaException, TException;
+      String writeIdList)  throws NoSuchObjectException, MetaException, TException;
 
   /**
    * Get aggregated column stats for a set of partitions.
@@ -3320,7 +3314,7 @@ public interface IMetaStoreClient {
 
   AggrStats getAggrColStatsFor(String catName, String dbName, String tblName,
                                List<String> colNames, List<String> partNames,
-                               long txnId, String writeIdList)
+                               String writeIdList)
       throws NoSuchObjectException, MetaException, TException;
   /**
    * Set table or partition column statistics.

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 9eb8424..b319e68 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1412,12 +1412,12 @@ public class ObjectStore implements RawStore, Configurable {
   public Table
   getTable(String catName, String dbName, String tableName)
       throws MetaException {
-    return getTable(catName, dbName, tableName, -1, null);
+    return getTable(catName, dbName, tableName, null);
   }
 
   @Override
   public Table getTable(String catName, String dbName, String tableName,
-                        long txnId, String writeIdList)
+                        String writeIdList)
       throws MetaException {
     boolean commited = false;
     Table tbl = null;
@@ -1441,7 +1441,7 @@ public class ObjectStore implements RawStore, Configurable {
           StatsSetupConst.setBasicStatsState(tbl.getParameters(), StatsSetupConst.FALSE);
           LOG.info("Removed COLUMN_STATS_ACCURATE from Table's parameters.");
         } else if (isTxn && tbl.getPartitionKeysSize() == 0) {
-          if (isCurrentStatsValidForTheQuery(mtable, txnId, writeIdList, false)) {
+          if (isCurrentStatsValidForTheQuery(mtable, writeIdList, false)) {
             tbl.setIsStatsCompliant(true);
           } else {
             tbl.setIsStatsCompliant(false);
@@ -2427,13 +2427,13 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public Partition getPartition(String catName, String dbName, String tableName,
       List<String> part_vals) throws NoSuchObjectException, MetaException {
-    return getPartition(catName, dbName, tableName, part_vals, -1, null);
+    return getPartition(catName, dbName, tableName, part_vals, null);
   }
 
   @Override
   public Partition getPartition(String catName, String dbName, String tableName,
                                 List<String> part_vals,
-                                long txnId, String writeIdList)
+                                String writeIdList)
       throws NoSuchObjectException, MetaException {
     openTransaction();
     MTable table = this.getMTable(catName, dbName, tableName);
@@ -2453,7 +2453,7 @@ public class ObjectStore implements RawStore, Configurable {
         StatsSetupConst.setBasicStatsState(part.getParameters(), StatsSetupConst.FALSE);
         LOG.info("Removed COLUMN_STATS_ACCURATE from Partition object's parameters.");
       } else if (writeIdList != null) {
-        if (isCurrentStatsValidForTheQuery(part, mpart.getWriteId(), txnId, writeIdList, false)) {
+        if (isCurrentStatsValidForTheQuery(part, mpart.getWriteId(), writeIdList, false)) {
           part.setIsStatsCompliant(true);
         } else {
           part.setIsStatsCompliant(false);
@@ -3022,7 +3022,7 @@ public class ObjectStore implements RawStore, Configurable {
         TableName.getQualified(catName, dbName, tableName), filter, cols);
     List<String> partitionNames = null;
     List<Partition> partitions = null;
-    Table tbl = getTable(catName, dbName, tableName, -1, null);
+    Table tbl = getTable(catName, dbName, tableName, null);
     try {
       // Get partitions by name - ascending or descending
       partitionNames = getPartitionNamesByFilter(catName, dbName, tableName, filter, ascending,
@@ -3156,7 +3156,7 @@ public class ObjectStore implements RawStore, Configurable {
         partValuesSelect.append("DISTINCT ");
       }
       List<FieldSchema> partitionKeys =
-          getTable(catName, dbName, tableName, -1, null).getPartitionKeys();
+          getTable(catName, dbName, tableName, null).getPartitionKeys();
       for (FieldSchema key : cols) {
         partValuesSelect.append(extractPartitionKey(key, partitionKeys)).append(", ");
       }
@@ -3238,7 +3238,7 @@ public class ObjectStore implements RawStore, Configurable {
     catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tableName = normalizeIdentifier(tableName);
-    Table table = getTable(catName, dbName, tableName, -1, null);
+    Table table = getTable(catName, dbName, tableName, null);
     if (table == null) {
       throw new NoSuchObjectException(TableName.getQualified(catName, dbName, tableName)
           + " table not found");
@@ -4092,7 +4092,7 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public void alterTable(String catName, String dbname, String name, Table newTable,
-      long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+      String queryValidWriteIds) throws InvalidObjectException, MetaException {
     boolean success = false;
     boolean registerCreationSignature = false;
     try {
@@ -4151,7 +4151,7 @@ public class ObjectStore implements RawStore, Configurable {
           StatsSetupConst.setBasicStatsState(oldt.getParameters(), StatsSetupConst.FALSE);
         } else if (queryValidWriteIds != null && (!isToTxn || newTable.getWriteId() > 0)) {
           // Check concurrent INSERT case and set false to the flag.
-          if (!isCurrentStatsValidForTheQuery(oldt, queryTxnId, queryValidWriteIds, true)) {
+          if (!isCurrentStatsValidForTheQuery(oldt, queryValidWriteIds, true)) {
             StatsSetupConst.setBasicStatsState(oldt.getParameters(), StatsSetupConst.FALSE);
             LOG.info("Removed COLUMN_STATS_ACCURATE from the parameters of the table " +
                     dbname + "." + name + ". will be made persistent.");
@@ -4231,7 +4231,7 @@ public class ObjectStore implements RawStore, Configurable {
    * @throws MetaException
    */
   private MColumnDescriptor alterPartitionNoTxn(String catName, String dbname, String name,
-    List<String> part_vals, Partition newPart, long queryTxnId, String queryValidWriteIds)
+    List<String> part_vals, Partition newPart, String queryValidWriteIds)
       throws InvalidObjectException, MetaException {
     catName = normalizeIdentifier(catName);
     name = normalizeIdentifier(name);
@@ -4276,7 +4276,7 @@ public class ObjectStore implements RawStore, Configurable {
         StatsSetupConst.setBasicStatsState(oldp.getParameters(), StatsSetupConst.FALSE);
       } else if (queryValidWriteIds != null && newPart.getWriteId() > 0) {
         // Check concurrent INSERT case and set false to the flag.
-        if (!isCurrentStatsValidForTheQuery(oldp, queryTxnId, queryValidWriteIds, true)) {
+        if (!isCurrentStatsValidForTheQuery(oldp, queryValidWriteIds, true)) {
           StatsSetupConst.setBasicStatsState(oldp.getParameters(), StatsSetupConst.FALSE);
           LOG.info("Removed COLUMN_STATS_ACCURATE from the parameters of the partition " +
                   dbname + "." + name + "." + oldp.getPartitionName() + " will be made persistent.");
@@ -4290,7 +4290,7 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public void alterPartition(String catName, String dbname, String name, List<String> part_vals,
-      Partition newPart, long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+      Partition newPart, String queryValidWriteIds) throws InvalidObjectException, MetaException {
     boolean success = false;
     Throwable e = null;
     try {
@@ -4299,7 +4299,7 @@ public class ObjectStore implements RawStore, Configurable {
         LOG.warn("Alter partitions with write ID called without transaction information");
       }
       MColumnDescriptor oldCd = alterPartitionNoTxn(
-          catName, dbname, name, part_vals, newPart, queryTxnId, queryValidWriteIds);
+          catName, dbname, name, part_vals, newPart, queryValidWriteIds);
       removeUnusedColumnDescriptor(oldCd);
       // commit the changes
       success = commitTransaction();
@@ -4322,7 +4322,7 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public void alterPartitions(String catName, String dbname, String name,
                               List<List<String>> part_vals, List<Partition> newParts,
-                              long writeId, long queryTxnId, String queryWriteIdList)
+                              long writeId, String queryWriteIdList)
                                   throws InvalidObjectException, MetaException {
     boolean success = false;
     Exception e = null;
@@ -4336,7 +4336,7 @@ public class ObjectStore implements RawStore, Configurable {
           tmpPart.setWriteId(writeId);
         }
         MColumnDescriptor oldCd = alterPartitionNoTxn(
-            catName, dbname, name, tmpPartVals, tmpPart, queryTxnId, queryWriteIdList);
+            catName, dbname, name, tmpPartVals, tmpPart, queryWriteIdList);
         if (oldCd != null) {
           oldCds.add(oldCd);
         }
@@ -6215,7 +6215,7 @@ public class ObjectStore implements RawStore, Configurable {
             boolean found = false;
             Table tabObj =
                 this.getTable(catName, hiveObject.getDbName(),
-                     hiveObject.getObjectName(), -1, null);
+                     hiveObject.getObjectName(), null);
             String partName = null;
             if (hiveObject.getPartValues() != null) {
               partName = Warehouse.makePartName(tabObj.getPartitionKeys(), hiveObject.getPartValues());
@@ -6249,7 +6249,7 @@ public class ObjectStore implements RawStore, Configurable {
           } else if (hiveObject.getObjectType() == HiveObjectType.COLUMN) {
 
             Table tabObj = this.getTable(catName, hiveObject.getDbName(), hiveObject
-                .getObjectName(), -1, null);
+                .getObjectName(), null);
             String partName = null;
             if (hiveObject.getPartValues() != null) {
               partName = Warehouse.makePartName(tabObj.getPartitionKeys(),
@@ -7771,7 +7771,7 @@ public class ObjectStore implements RawStore, Configurable {
       query
           .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, int t4," +
               "java.lang.String t5");
-      Table tbl = getTable(catName, dbName, tblName, -1, null); // Make sure dbName and tblName are valid.
+      Table tbl = getTable(catName, dbName, tblName, null); // Make sure dbName and tblName are valid.
       if (null == tbl) {
         throw new UnknownTableException("Table: " + tblName + " is not found.");
       }
@@ -7797,7 +7797,7 @@ public class ObjectStore implements RawStore, Configurable {
     Table tbl = null;
     try{
     openTransaction();
-    tbl = getTable(catName, dbName, tblName, -1, null); // Make sure dbName and tblName are valid.
+    tbl = getTable(catName, dbName, tblName, null); // Make sure dbName and tblName are valid.
     if(null == tbl) {
       throw new UnknownTableException("Table: "+ tblName + " is not found.");
     }
@@ -8409,7 +8409,7 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public boolean updateTableColumnStatistics(ColumnStatistics colStats,
-      long txnId, String validWriteIds, long writeId)
+      String validWriteIds, long writeId)
     throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
     boolean committed = false;
 
@@ -8455,7 +8455,7 @@ public class ObjectStore implements RawStore, Configurable {
           if (errorMsg != null) {
             throw new MetaException(errorMsg);
           }
-          if (!isCurrentStatsValidForTheQuery(oldt, txnId, validWriteIds, true)) {
+          if (!isCurrentStatsValidForTheQuery(oldt, validWriteIds, true)) {
             // Make sure we set the flag to invalid regardless of the current value.
             StatsSetupConst.setBasicStatsState(newParams, StatsSetupConst.FALSE);
             LOG.info("Removed COLUMN_STATS_ACCURATE from the parameters of the table "
@@ -8503,7 +8503,7 @@ public class ObjectStore implements RawStore, Configurable {
 
   @Override
   public boolean updatePartitionColumnStatistics(ColumnStatistics colStats, List<String> partVals,
-      long txnId, String validWriteIds, long writeId)
+      String validWriteIds, long writeId)
           throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
     boolean committed = false;
 
@@ -8549,7 +8549,7 @@ public class ObjectStore implements RawStore, Configurable {
           if (errorMsg != null) {
             throw new MetaException(errorMsg);
           }
-          if (!isCurrentStatsValidForTheQuery(mPartition, txnId, validWriteIds, true)) {
+          if (!isCurrentStatsValidForTheQuery(mPartition, validWriteIds, true)) {
             // Make sure we set the flag to invalid regardless of the current value.
             StatsSetupConst.setBasicStatsState(newParams, StatsSetupConst.FALSE);
             LOG.info("Removed COLUMN_STATS_ACCURATE from the parameters of the partition "
@@ -8660,7 +8660,6 @@ public class ObjectStore implements RawStore, Configurable {
       String dbName,
       String tableName,
       List<String> colNames,
-      long txnId,
       String writeIdList) throws MetaException, NoSuchObjectException {
     // If the current stats in the metastore doesn't comply with
     // the isolation level of the query, set No to the compliance flag.
@@ -8668,7 +8667,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (writeIdList != null) {
       MTable table = this.getMTable(catName, dbName, tableName);
       isCompliant = !TxnUtils.isTransactionalTable(table.getParameters())
-        || (areTxnStatsSupported && isCurrentStatsValidForTheQuery(table, txnId, writeIdList, false));
+        || (areTxnStatsSupported && isCurrentStatsValidForTheQuery(table, writeIdList, false));
     }
     ColumnStatistics stats = getTableColumnStatisticsInternal(
         catName, dbName, tableName, colNames, true, true);
@@ -8730,7 +8729,7 @@ public class ObjectStore implements RawStore, Configurable {
   public List<ColumnStatistics> getPartitionColumnStatistics(
       String catName, String dbName, String tableName,
       List<String> partNames, List<String> colNames,
-      long txnId, String writeIdList)
+      String writeIdList)
       throws MetaException, NoSuchObjectException {
     if (partNames == null && partNames.isEmpty()) {
       return null;
@@ -8748,11 +8747,11 @@ public class ObjectStore implements RawStore, Configurable {
           MPartition mpart = getMPartition(catName, dbName, tableName,
               Warehouse.getPartValuesFromPartName(cs.getStatsDesc().getPartName()));
           if (mpart == null
-              || !isCurrentStatsValidForTheQuery(mpart, txnId, writeIdList, false)) {
+              || !isCurrentStatsValidForTheQuery(mpart, writeIdList, false)) {
             if (mpart != null) {
               LOG.debug("The current metastore transactional partition column statistics for {}.{}.{} "
                 + "(write ID {}) are not valid for current query ({} {})", dbName, tableName,
-                mpart.getPartitionName(), mpart.getWriteId(), txnId, writeIdList);
+                mpart.getPartitionName(), mpart.getWriteId(), writeIdList);
             }
             cs.setIsStatsCompliant(false);
           } else {
@@ -8815,7 +8814,7 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public AggrStats get_aggr_stats_for(String catName, String dbName, String tblName,
       final List<String> partNames, final List<String> colNames,
-      long txnId, String writeIdList) throws MetaException, NoSuchObjectException {
+      String writeIdList) throws MetaException, NoSuchObjectException {
     // If the current stats in the metastore doesn't comply with
     // the isolation level of the query, return null.
     if (writeIdList != null) {
@@ -8836,7 +8835,7 @@ public class ObjectStore implements RawStore, Configurable {
       for (String partName : partNames) {
         MPartition mpart = getMPartition(
             catName, dbName, tblName, Warehouse.getPartValuesFromPartName(partName));
-        if (!isCurrentStatsValidForTheQuery(mpart, txnId, writeIdList, false)) {
+        if (!isCurrentStatsValidForTheQuery(mpart, writeIdList, false)) {
           LOG.debug("The current metastore transactional partition column statistics " +
                   "for " + dbName + "." + tblName + "." + mpart.getPartitionName() + " is not valid " +
                   "for the current query.");
@@ -12419,10 +12418,10 @@ public class ObjectStore implements RawStore, Configurable {
    * @param queryWriteId           writeId of the query
    * @Precondition   "tbl" should be retrieved from the TBLS table.
    */
-  private boolean isCurrentStatsValidForTheQuery(MTable tbl, long queryTxnId, String queryValidWriteIdList,
+  private boolean isCurrentStatsValidForTheQuery(MTable tbl, String queryValidWriteIdList,
       boolean isCompleteStatsWriter) throws MetaException {
     return isCurrentStatsValidForTheQuery(conf, tbl.getDatabase().getName(), tbl.getTableName(),
-        tbl.getParameters(), tbl.getWriteId(), queryTxnId, queryValidWriteIdList, isCompleteStatsWriter);
+        tbl.getParameters(), tbl.getWriteId(), queryValidWriteIdList, isCompleteStatsWriter);
   }
 
   /**
@@ -12439,30 +12438,30 @@ public class ObjectStore implements RawStore, Configurable {
    * @param queryValidWriteIdList  valid writeId list of the query
    * @Precondition   "part" should be retrieved from the PARTITIONS table.
    */
-  private boolean isCurrentStatsValidForTheQuery(MPartition part, long queryTxnId,
+  private boolean isCurrentStatsValidForTheQuery(MPartition part,
       String queryValidWriteIdList, boolean isCompleteStatsWriter)
       throws MetaException {
     return isCurrentStatsValidForTheQuery(conf, part.getTable().getDatabase().getName(),
         part.getTable().getTableName(), part.getParameters(), part.getWriteId(),
-        queryTxnId, queryValidWriteIdList, isCompleteStatsWriter);
+        queryValidWriteIdList, isCompleteStatsWriter);
   }
 
-  private boolean isCurrentStatsValidForTheQuery(Partition part, long partWriteId, long queryTxnId,
+  private boolean isCurrentStatsValidForTheQuery(Partition part, long partWriteId,
       String queryValidWriteIdList, boolean isCompleteStatsWriter)
       throws MetaException {
     return isCurrentStatsValidForTheQuery(conf, part.getDbName(), part.getTableName(),
-        part.getParameters(), partWriteId, queryTxnId, queryValidWriteIdList, isCompleteStatsWriter);
+        part.getParameters(), partWriteId, queryValidWriteIdList, isCompleteStatsWriter);
   }
 
   // TODO: move to somewhere else
   public static boolean isCurrentStatsValidForTheQuery(Configuration conf, String dbName,
-      String tblName, Map<String, String> statsParams, long statsWriteId, long queryTxnId,
+      String tblName, Map<String, String> statsParams, long statsWriteId,
       String queryValidWriteIdList, boolean isCompleteStatsWriter) throws MetaException {
 
     // Note: can be changed to debug/info to verify the calls.
     // TODO## change this to debug when merging
-    LOG.info("isCurrentStatsValidForTheQuery with stats write ID {}; query {}, {}; writer: {} params {}",
-        statsWriteId, queryTxnId, queryValidWriteIdList, isCompleteStatsWriter, statsParams);
+    LOG.info("isCurrentStatsValidForTheQuery with stats write ID {}; query {}; writer: {} params {}",
+        statsWriteId, queryValidWriteIdList, isCompleteStatsWriter, statsParams);
     // return true since the stats does not seem to be transactional.
     if (statsWriteId < 1) {
       return true;
@@ -12495,15 +12494,6 @@ public class ObjectStore implements RawStore, Configurable {
       }
     }
 
-    if (queryTxnId < 1) {
-      return false; // The caller is outside of a txn; no need to check the same-txn case.
-    }
-
-    // This assumes that all writes within the same txn are sequential and can see each other.
-    // TODO## Not clear if we need this check; each next write should have the previous
-    //        one in its writeIdList; verify w/Eugene.
-    long statsTxnId = HiveMetaStore.HMSHandler.getMsThreadTxnHandler(conf).getTxnIdForWriteId(
-        dbName, tblName, statsWriteId);
-    return (statsTxnId == queryTxnId);
+    return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index 95e8445..46082a5 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -221,7 +221,7 @@ public interface RawStore extends Configurable {
    * @throws MetaException something went wrong in the RDBMS
    */
   Table getTable(String catalogName, String dbName, String tableName,
-                 long txnId, String writeIdList) throws MetaException;
+                 String writeIdList) throws MetaException;
 
   /**
    * Add a partition.
@@ -289,7 +289,7 @@ public interface RawStore extends Configurable {
    */
   Partition getPartition(String catName, String dbName, String tableName,
                          List<String> part_vals,
-                         long txnId, String writeIdList)
+                         String writeIdList)
       throws MetaException, NoSuchObjectException;
 
   /**
@@ -362,7 +362,7 @@ public interface RawStore extends Configurable {
    * @throws MetaException something went wrong, usually in the RDBMS or storage.
    */
   void alterTable(String catName, String dbname, String name, Table newTable,
-      long queryTxnId, String queryValidWriteIds)
+      String queryValidWriteIds)
       throws InvalidObjectException, MetaException;
 
   /**
@@ -503,7 +503,7 @@ public interface RawStore extends Configurable {
    * @throws MetaException error accessing the RDBMS.
    */
   void alterPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
-      Partition new_part, long queryTxnId, String queryValidWriteIds)
+      Partition new_part, String queryValidWriteIds)
           throws InvalidObjectException, MetaException;
 
   /**
@@ -524,7 +524,7 @@ public interface RawStore extends Configurable {
    */
   void alterPartitions(String catName, String db_name, String tbl_name,
       List<List<String>> part_vals_list, List<Partition> new_parts, long writeId,
-      long queryTxnId, String queryValidWriteIds)
+      String queryValidWriteIds)
       throws InvalidObjectException, MetaException;
 
   /**
@@ -864,7 +864,7 @@ public interface RawStore extends Configurable {
    * @throws InvalidObjectException the stats object is invalid
    * @throws InvalidInputException unable to record the stats for the table
    */
-  boolean updateTableColumnStatistics(ColumnStatistics colStats, long txnId, String validWriteIds, long writeId)
+  boolean updateTableColumnStatistics(ColumnStatistics colStats, String validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException;
 
   /** Persists the given column statistics object to the metastore
@@ -878,7 +878,7 @@ public interface RawStore extends Configurable {
    * @throws TException
    */
   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;
 
   /**
@@ -912,7 +912,7 @@ public interface RawStore extends Configurable {
    */
   ColumnStatistics getTableColumnStatistics(
     String catName, String dbName, String tableName,
-    List<String> colName, long txnId, String writeIdList)
+    List<String> colName, String writeIdList)
       throws MetaException, NoSuchObjectException;
 
   /**
@@ -946,7 +946,7 @@ public interface RawStore extends Configurable {
   List<ColumnStatistics> getPartitionColumnStatistics(
       String catName, String dbName, String tblName,
       List<String> partNames, List<String> colNames,
-      long txnId, String writeIdList)
+      String writeIdList)
       throws MetaException, NoSuchObjectException;
 
   /**
@@ -1208,7 +1208,7 @@ public interface RawStore extends Configurable {
    */
   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, NoSuchObjectException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 9bee0db..dd705a5 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -825,20 +825,20 @@ public class CachedStore implements RawStore, Configurable {
 
   @Override
   public Table getTable(String catName, String dbName, String tblName) throws MetaException {
-    return getTable(catName, dbName, tblName, -1, null);
+    return getTable(catName, dbName, tblName, null);
   }
 
   // TODO: if writeIdList is not null, check isolation level compliance for SVS,
   // possibly with getTableFromCache() with table snapshot in cache.
   @Override
   public Table getTable(String catName, String dbName, String tblName,
-                        long txnId, String writeIdList)
+                        String writeIdList)
       throws MetaException {
     catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
     if (!shouldCacheTable(catName, dbName, tblName)) {
-      return rawStore.getTable(catName, dbName, tblName, txnId,writeIdList);
+      return rawStore.getTable(catName, dbName, tblName, writeIdList);
     }
     Table tbl = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (tbl == null || writeIdList != null) {
@@ -847,7 +847,7 @@ public class CachedStore implements RawStore, Configurable {
       // let's move this table to the top of tblNamesBeingPrewarmed stack,
       // so that it gets loaded to the cache faster and is available for subsequent requests
       tblsPendingPrewarm.prioritizeTableForPrewarm(tblName);
-      return rawStore.getTable(catName, dbName, tblName, txnId, writeIdList);
+      return rawStore.getTable(catName, dbName, tblName, writeIdList);
     }
     if (tbl != null) {
       tbl.unsetPrivileges();
@@ -910,26 +910,26 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public Partition getPartition(String catName, String dbName, String tblName, List<String> part_vals)
       throws MetaException, NoSuchObjectException {
-    return getPartition(catName, dbName, tblName, part_vals, -1, null);
+    return getPartition(catName, dbName, tblName, part_vals, null);
   }
 
   // TODO: the same as getTable()
   @Override
   public Partition getPartition(String catName, String dbName, String tblName,
-                                List<String> part_vals, long txnId, String writeIdList)
+                                List<String> part_vals, String writeIdList)
       throws MetaException, NoSuchObjectException {
     catName = normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
     if (!shouldCacheTable(catName, dbName, tblName)) {
       return rawStore.getPartition(
-          catName, dbName, tblName, part_vals, txnId, writeIdList);
+          catName, dbName, tblName, part_vals, writeIdList);
     }
     Partition part = sharedCache.getPartitionFromCache(catName, dbName, tblName, part_vals);
     if (part == null || writeIdList != null) {
       // The table containing the partition is not yet loaded in cache
       return rawStore.getPartition(
-          catName, dbName, tblName, part_vals, txnId, writeIdList);
+          catName, dbName, tblName, part_vals, writeIdList);
     }
     return part;
   }
@@ -1011,8 +1011,8 @@ public class CachedStore implements RawStore, Configurable {
 
   @Override
   public void alterTable(String catName, String dbName, String tblName, Table newTable,
-      long txnId, String validWriteIds) throws InvalidObjectException, MetaException {
-    rawStore.alterTable(catName, dbName, tblName, newTable, txnId, validWriteIds);
+      String validWriteIds) throws InvalidObjectException, MetaException {
+    rawStore.alterTable(catName, dbName, tblName, newTable, validWriteIds);
     catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tblName = normalizeIdentifier(tblName);
@@ -1162,9 +1162,9 @@ public class CachedStore implements RawStore, Configurable {
 
   @Override
   public void alterPartition(String catName, String dbName, String tblName, List<String> partVals,
-                             Partition newPart, long queryTxnId, String queryValidWriteIds)
+                             Partition newPart, String queryValidWriteIds)
                                  throws InvalidObjectException, MetaException {
-    rawStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryTxnId, queryValidWriteIds);
+    rawStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryValidWriteIds);
     catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tblName = normalizeIdentifier(tblName);
@@ -1177,10 +1177,10 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public void alterPartitions(String catName, String dbName, String tblName,
                               List<List<String>> partValsList, List<Partition> newParts,
-                              long writeId, long txnId, String validWriteIds)
+                              long writeId, String validWriteIds)
       throws InvalidObjectException, MetaException {
     rawStore.alterPartitions(
-        catName, dbName, tblName, partValsList, newParts, writeId, txnId, validWriteIds);
+        catName, dbName, tblName, partValsList, newParts, writeId, validWriteIds);
     catName = normalizeIdentifier(catName);
     dbName = normalizeIdentifier(dbName);
     tblName = normalizeIdentifier(tblName);
@@ -1599,9 +1599,9 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public boolean updateTableColumnStatistics(ColumnStatistics colStats, long txnId, String validWriteIds, long writeId)
+  public boolean updateTableColumnStatistics(ColumnStatistics colStats, String validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-    boolean succ = rawStore.updateTableColumnStatistics(colStats, txnId, validWriteIds, writeId);
+    boolean succ = rawStore.updateTableColumnStatistics(colStats, validWriteIds, writeId);
     if (succ) {
       String catName = colStats.getStatsDesc().isSetCatName() ?
           normalizeIdentifier(colStats.getStatsDesc().getCatName()) :
@@ -1631,27 +1631,27 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public ColumnStatistics getTableColumnStatistics(String catName, String dbName, String tblName,
       List<String> colNames) throws MetaException, NoSuchObjectException {
-    return getTableColumnStatistics(catName, dbName, tblName, colNames, -1, null);
+    return getTableColumnStatistics(catName, dbName, tblName, colNames, null);
   }
 
   // TODO: the same as getTable()
   @Override
   public ColumnStatistics getTableColumnStatistics(
       String catName, String dbName, String tblName, List<String> colNames,
-      long txnId, String writeIdList)
+      String writeIdList)
       throws MetaException, NoSuchObjectException {
     catName = StringUtils.normalizeIdentifier(catName);
     dbName = StringUtils.normalizeIdentifier(dbName);
     tblName = StringUtils.normalizeIdentifier(tblName);
     if (!shouldCacheTable(catName, dbName, tblName)) {
       return rawStore.getTableColumnStatistics(
-          catName, dbName, tblName, colNames, txnId, writeIdList);
+          catName, dbName, tblName, colNames, writeIdList);
     }
     Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null || writeIdList != null) {
       // The table is not yet loaded in cache
       return rawStore.getTableColumnStatistics(
-          catName, dbName, tblName, colNames, txnId, writeIdList);
+          catName, dbName, tblName, colNames, writeIdList);
     }
     ColumnStatisticsDesc csd = new ColumnStatisticsDesc(true, dbName, tblName);
     List<ColumnStatisticsObj> colStatObjs =
@@ -1678,9 +1678,9 @@ public class CachedStore implements RawStore, Configurable {
 
   @Override
   public boolean updatePartitionColumnStatistics(ColumnStatistics colStats, List<String> partVals,
-      long txnId, String validWriteIds, long writeId)
+      String validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException, InvalidInputException {
-    boolean succ = rawStore.updatePartitionColumnStatistics(colStats, partVals, txnId, validWriteIds, writeId);
+    boolean succ = rawStore.updatePartitionColumnStatistics(colStats, partVals, validWriteIds, writeId);
     if (succ) {
       String catName = colStats.getStatsDesc().isSetCatName() ?
           normalizeIdentifier(colStats.getStatsDesc().getCatName()) : DEFAULT_CATALOG_NAME;
@@ -1712,10 +1712,10 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public List<ColumnStatistics> getPartitionColumnStatistics(
       String catName, String dbName, String tblName, List<String> partNames,
-      List<String> colNames, long txnId, String writeIdList)
+      List<String> colNames, String writeIdList)
       throws MetaException, NoSuchObjectException {
     return rawStore.getPartitionColumnStatistics(
-        catName, dbName, tblName, partNames, colNames, txnId, writeIdList);
+        catName, dbName, tblName, partNames, colNames, writeIdList);
   }
 
   @Override
@@ -1739,14 +1739,14 @@ public class CachedStore implements RawStore, Configurable {
   @Override
   public AggrStats get_aggr_stats_for(String catName, String dbName, String tblName, List<String> partNames,
       List<String> colNames) throws MetaException, NoSuchObjectException {
-    return get_aggr_stats_for(catName, dbName, tblName, partNames, colNames, -1, null);
+    return get_aggr_stats_for(catName, dbName, tblName, partNames, colNames, null);
   }
 
   @Override
   // TODO: the same as getTable() for transactional stats.
   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, NoSuchObjectException {
     List<ColumnStatisticsObj> colStats;
     catName = normalizeIdentifier(catName);
@@ -1754,13 +1754,13 @@ public class CachedStore implements RawStore, Configurable {
     tblName = StringUtils.normalizeIdentifier(tblName);
     if (!shouldCacheTable(catName, dbName, tblName)) {
       rawStore.get_aggr_stats_for(
-          catName, dbName, tblName, partNames, colNames, txnId, writeIdList);
+          catName, dbName, tblName, partNames, colNames, writeIdList);
     }
     Table table = sharedCache.getTableFromCache(catName, dbName, tblName);
     if (table == null || writeIdList != null) {
       // The table is not yet loaded in cache
       return rawStore.get_aggr_stats_for(
-          catName, dbName, tblName, partNames, colNames, txnId, writeIdList);
+          catName, dbName, tblName, partNames, colNames, writeIdList);
     }
     List<String> allPartNames = rawStore.listPartitionNames(catName, dbName, tblName, (short) -1);
     if (partNames.size() == allPartNames.size()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index a5e6918..fb14536 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -248,9 +248,9 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public Table getTable(String catName, String dbName, String tableName, long txnId, String writeIdList)
+  public Table getTable(String catName, String dbName, String tableName, String writeIdList)
       throws MetaException {
-    return objectStore.getTable(catName, dbName, tableName, txnId, writeIdList);
+    return objectStore.getTable(catName, dbName, tableName, writeIdList);
   }
 
   @Override
@@ -267,9 +267,9 @@ public class DummyRawStoreControlledCommit 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
@@ -293,9 +293,9 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
 
   @Override
   public void alterTable(String catName, String dbName, String name, Table newTable,
-      long queryTxnId, String queryValidWriteIds)
+      String queryValidWriteIds)
       throws InvalidObjectException, MetaException {
-    objectStore.alterTable(catName, dbName, name, newTable, queryTxnId, queryValidWriteIds);
+    objectStore.alterTable(catName, dbName, name, newTable, queryValidWriteIds);
   }
 
   @Override
@@ -358,16 +358,16 @@ public class DummyRawStoreControlledCommit 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 {
-    objectStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryTxnId, queryValidWriteIds);
+      Partition newPart, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+    objectStore.alterPartition(catName, dbName, tblName, partVals, newPart, queryValidWriteIds);
   }
 
   @Override
   public void alterPartitions(String catName, String dbName, String tblName,
       List<List<String>> partValsList, List<Partition> newParts,
-      long writeId, long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+      long writeId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
     objectStore.alterPartitions(
-        catName, dbName, tblName, partValsList, newParts, writeId, queryTxnId, queryValidWriteIds);
+        catName, dbName, tblName, partValsList, newParts, writeId, queryValidWriteIds);
   }
 
   @Override
@@ -671,10 +671,10 @@ public class DummyRawStoreControlledCommit 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);
+        catName, dbName, tableName, colNames, writeIdList);
   }
 
   @Override
@@ -694,18 +694,18 @@ public class DummyRawStoreControlledCommit 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
@@ -772,10 +772,10 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   @Override
   public List<ColumnStatistics> getPartitionColumnStatistics(
       String catName, String dbName, String tblName, List<String> partNames,
-      List<String> colNames, long txnId, String writeIdList)
+      List<String> colNames, String writeIdList)
       throws MetaException, NoSuchObjectException {
     return objectStore.getPartitionColumnStatistics(
-             catName, dbName, tblName  , colNames, partNames, txnId, writeIdList);
+             catName, dbName, tblName  , colNames, partNames, writeIdList);
   }
 
   @Override
@@ -850,7 +850,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   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, NoSuchObjectException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 8270f6a..52785a6 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -245,7 +245,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public Table getTable(String catalogName, String dbName, String tableName,
-                        long txnid, String writeIdList) throws MetaException {
+                        String  writeIdList) throws MetaException {
     return null;
   }
 
@@ -264,7 +264,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public Partition getPartition(String catName, String dbName, String tableName, List<String> part_vals,
-                                long txnid, String writeIdList)
+                                String  writeIdList)
       throws MetaException, NoSuchObjectException {
     return null;
   }
@@ -290,7 +290,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @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 {
   }
 
@@ -359,13 +359,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public void alterPartition(String catName, String db_name, String tbl_name, List<String> part_vals,
-      Partition new_part, long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+      Partition new_part, String queryValidWriteIds) throws InvalidObjectException, MetaException {
   }
 
   @Override
   public void alterPartitions(String catName, String db_name, String tbl_name,
-                              List<List<String>> part_vals_list, List<Partition> new_parts,
-                              long writeId, long queryTxnId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
+      List<List<String>> part_vals_list, List<Partition> new_parts,
+      long writeId, String queryValidWriteIds) throws InvalidObjectException, MetaException {
   }
 
   @Override
@@ -720,7 +720,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   @Override
   public ColumnStatistics getTableColumnStatistics(
       String catName, String dbName, String tableName, List<String> colName,
-      long txnid, String writeIdList)
+      String  writeIdList)
       throws MetaException, NoSuchObjectException {
     return null;
   }
@@ -744,14 +744,14 @@ public class DummyRawStoreForJdoConnection implements RawStore {
 
   @Override
   public boolean updateTableColumnStatistics(ColumnStatistics statsObj,
-      long txnId, String validWriteIds, long writeId)
+      String  validWriteIds, long writeId)
       throws NoSuchObjectException, MetaException, InvalidObjectException {
     return false;
   }
 
   @Override
   public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj,List<String> partVals,
-      long txnId, String validWriteIds, long writeId)
+      String  validWriteIds, long writeId)
     throws NoSuchObjectException, MetaException, InvalidObjectException {
     return false;
   }
@@ -779,7 +779,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   @Override
   public List<ColumnStatistics> getPartitionColumnStatistics(
       String catName, String dbName, String tblName, List<String> partNames,
-      List<String> colNames, long txnid, String writeIdList)
+      List<String> colNames, String  writeIdList)
       throws MetaException, NoSuchObjectException {
     return Collections.emptyList();
   }
@@ -850,7 +850,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   @Override
   public AggrStats get_aggr_stats_for(
       String catName, String dbName, String tblName, List<String> partNames,
-      List<String> colNames, long txnid, String writeIdList)
+      List<String> colNames, String  writeIdList)
       throws MetaException, NoSuchObjectException {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index bc04e06..34055d2 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -1428,17 +1428,6 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
     return fastpath ? t : deepCopy(filterHook.filterTable(t));
   }
 
-  @Override
-  public Table getTable(String dbName, String tableName, long txnId, String validWriteIdList)
-      throws MetaException, TException, NoSuchObjectException {
-    GetTableRequest req = new GetTableRequest(dbName, tableName);
-    req.setCapabilities(version);
-    req.setTxnId(txnId);
-    req.setValidWriteIdList(validWriteIdList);
-    Table t = client.get_table_req(req).getTable();
-    return fastpath ? t : deepCopy(filterHook.filterTable(t));
-  }
-
   /** {@inheritDoc} */
   @Override
   public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
@@ -1638,14 +1627,13 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public void alter_partitions(String dbName, String tblName, List<Partition> newParts,
                                EnvironmentContext environmentContext,
-                               long txnId, String writeIdList, long writeId)
+                               String writeIdList, long writeId)
       throws InvalidOperationException, MetaException, TException {
     AlterPartitionsRequest req = new AlterPartitionsRequest();
     req.setDbName(dbName);
     req.setTableName(tblName);
     req.setPartitions(newParts);
     req.setEnvironmentContext(environmentContext);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(writeIdList);
     client.alter_partitions_req(req);
   }
@@ -1758,10 +1746,9 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   @Override
   public List<ColumnStatisticsObj> getTableColumnStatistics(
-      String dbName, String tableName, List<String> colNames, long txnId, String validWriteIdList)
+      String dbName, String tableName, List<String> colNames, String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     TableStatsRequest tsr = new TableStatsRequest(dbName, tableName, colNames);
-    tsr.setTxnId(txnId);
     tsr.setValidWriteIdList(validWriteIdList);
 
     return client.get_table_statistics_req(tsr).getTableStats();
@@ -1779,10 +1766,9 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String dbName, String tableName, List<String> partNames,
-      List<String> colNames, long txnId, String validWriteIdList)
+      List<String> colNames, String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     PartitionsStatsRequest psr = new PartitionsStatsRequest(dbName, tableName, colNames, partNames);
-    psr.setTxnId(txnId);
     psr.setValidWriteIdList(validWriteIdList);
     return client.get_partitions_statistics_req(
         psr).getPartStats();
@@ -2651,14 +2637,13 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public AggrStats getAggrColStatsFor(
       String dbName, String tblName, List<String> colNames,
-      List<String> partName, long txnId, String writeIdList)
+      List<String> partName, String writeIdList)
       throws NoSuchObjectException, MetaException, TException {
     if (colNames.isEmpty() || partName.isEmpty()) {
       LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval on client side.");
       return new AggrStats(new ArrayList<>(),0); // Nothing to aggregate
     }
     PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, colNames, partName);
-    req.setTxnId(txnId);
     req.setValidWriteIdList(writeIdList);
     return client.get_aggr_stats_for(req);
   }
@@ -3072,7 +3057,7 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   @Override
   public Table getTable(String catName, String dbName, String tableName,
-                        long txnId, String validWriteIdList) throws TException {
+                        String validWriteIdList) throws TException {
     throw new UnsupportedOperationException();
   }
 
@@ -3304,14 +3289,14 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   public void alter_partitions(String catName, String dbName, String tblName,
                                List<Partition> newParts,
                                EnvironmentContext environmentContext,
-                               long txnId, String writeIdList, long writeId) throws
+                               String writeIdList, long writeId) throws
       InvalidOperationException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
 
   @Override
   public void renamePartition(String catName, String dbname, String tableName,
-      List<String> part_vals, Partition newPart, long txnId, String validWriteIds)
+      List<String> part_vals, Partition newPart, String validWriteIds)
           throws InvalidOperationException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
@@ -3339,7 +3324,7 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public List<ColumnStatisticsObj> getTableColumnStatistics(
       String catName, String dbName, String tableName, List<String> colNames,
-      long txnId, String validWriteIdList)
+      String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
@@ -3357,7 +3342,7 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
       String catName, String dbName, String tableName, List<String> partNames,
-      List<String> colNames, long txnId, String validWriteIdList)
+      List<String> colNames, String validWriteIdList)
       throws NoSuchObjectException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
@@ -3412,7 +3397,7 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   @Override
   public AggrStats getAggrColStatsFor(String catName, String dbName, String tblName,
                                       List<String> colNames, List<String> partNames,
-                                      long txnId, String writeIdList)
+                                      String writeIdList)
       throws NoSuchObjectException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
@@ -3525,21 +3510,21 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
 
   @Override
   public void alter_table(String catName, String databaseName, String tblName, Table table,
-      EnvironmentContext environmentContext, long txnId, String validWriteIdList)
+      EnvironmentContext environmentContext, String validWriteIdList)
       throws InvalidOperationException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
 
   @Override
   public void alter_partition(String dbName, String tblName, Partition newPart,
-      EnvironmentContext environmentContext, long txnId, String writeIdList)
+      EnvironmentContext environmentContext, String writeIdList)
       throws InvalidOperationException, MetaException, TException {
     throw new UnsupportedOperationException();
   }
 
   @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 {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
index c5977b2..481d1d2 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
@@ -136,9 +136,8 @@ public class InjectableBehaviourObjectStore extends ObjectStore {
   }
 
   @Override
-  public Table getTable(String catName, String dbName, String tableName,
-      long txnId, String writeIdList) throws MetaException {
-    return getTableModifier.apply(super.getTable(catName, dbName, tableName, txnId, writeIdList));
+  public Table getTable(String catName, String dbName, String tableName, String writeIdList) throws MetaException {
+    return getTableModifier.apply(super.getTable(catName, dbName, tableName, writeIdList));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
index 533cabb..8816480 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveAlterHandler.java
@@ -60,7 +60,7 @@ public class TestHiveAlterHandler {
         getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3"));
     HiveAlterHandler handler = new HiveAlterHandler();
     handler.setConf(conf);
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, -1, null);
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, null);
   }
 
   @Test
@@ -85,7 +85,7 @@ public class TestHiveAlterHandler {
     RawStore msdb = Mockito.mock(RawStore.class);
     HiveAlterHandler handler = new HiveAlterHandler();
     handler.setConf(conf);
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, -1, null);
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, null);
     Mockito.verify(msdb, Mockito.times(1)).getTableColumnStatistics(
         getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4")
     );
@@ -115,7 +115,7 @@ public class TestHiveAlterHandler {
         getDefaultCatalog(conf), oldTable.getDbName(), oldTable.getTableName(), Arrays.asList("col1", "col2", "col3", "col4"));
     HiveAlterHandler handler = new HiveAlterHandler();
     handler.setConf(conf);
-    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, -1, null);
+    handler.alterTableUpdateTableColumnStats(msdb, oldTable, newTable, null, null);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 995271a..e53ad77 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -245,7 +245,7 @@ public class TestObjectStore {
     newTbl1.setOwner("role1");
     newTbl1.setOwnerType(PrincipalType.ROLE);
 
-    objectStore.alterTable(DEFAULT_CATALOG_NAME, DB1, TABLE1, newTbl1, -1, null);
+    objectStore.alterTable(DEFAULT_CATALOG_NAME, DB1, TABLE1, newTbl1, null);
     tables = objectStore.getTables(DEFAULT_CATALOG_NAME, DB1, "new*");
     Assert.assertEquals(1, tables.size());
     Assert.assertEquals("new" + TABLE1, tables.get(0));
@@ -559,7 +559,7 @@ public class TestObjectStore {
         ColumnStatisticsObj partStats = new ColumnStatisticsObj("test_part_col", "int", data);
         statsObjList.add(partStats);
 
-        objectStore.updatePartitionColumnStatistics(stats, part.getValues(), -1, null, -1);
+        objectStore.updatePartitionColumnStatistics(stats, part.getValues(), null, -1);
       }
     }
     if (withPrivileges) {

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
index 01a8f81..36f91eb 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
@@ -175,7 +175,7 @@ public class TestOldSchema {
       data.setLongStats(dcsd);
       obj.setStatsData(data);
       cs.addToStatsObj(obj);
-      store.updatePartitionColumnStatistics(cs, partVal, -1, null, -1);
+      store.updatePartitionColumnStatistics(cs, partVal, null, -1);
 
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index e4854f9..bb20d9f 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@ -232,7 +232,7 @@ public class TestCachedStore {
     tblOwner = "role1";
     tbl.setOwner(tblOwner);
     tbl.setOwnerType(PrincipalType.ROLE);
-    objectStore.alterTable(DEFAULT_CATALOG_NAME, dbName, tblName, tbl, -1, null);
+    objectStore.alterTable(DEFAULT_CATALOG_NAME, dbName, tblName, tbl, null);
     tbl = objectStore.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
 
     Assert.assertEquals("Owner of the table did not change.", tblOwner, tbl.getOwner());
@@ -338,7 +338,7 @@ public class TestCachedStore {
     Partition ptn1Atl =
         new Partition(Arrays.asList(ptnColVal1Alt), dbName, tblName, 0, 0, tbl.getSd(), partParams);
     ptn1Atl.setCatName(DEFAULT_CATALOG_NAME);
-    objectStore.alterPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1), ptn1Atl, -1, null);
+    objectStore.alterPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1), ptn1Atl, null);
     ptn1Atl = objectStore.getPartition(DEFAULT_CATALOG_NAME, dbName, tblName, Arrays.asList(ptnColVal1Alt));
 
     // Drop an existing partition ("bbb") via ObjectStore
@@ -455,7 +455,7 @@ public class TestCachedStore {
     stats.setStatsObj(colStatObjs);
 
     // Save to DB
-    objectStore.updateTableColumnStatistics(stats, -1, null, -1);
+    objectStore.updateTableColumnStatistics(stats, null, -1);
 
     // Prewarm CachedStore
     CachedStore.setCachePrewarmedState(false);
@@ -720,8 +720,8 @@ public class TestCachedStore {
     stats.setStatsDesc(statsDesc);
     stats.setStatsObj(colStatObjs);
 
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, -1, null, -1);
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, -1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, null, -1);
 
     List<String> colNames = new ArrayList<>();
     colNames.add(colName);
@@ -790,10 +790,10 @@ public class TestCachedStore {
     stats.setStatsDesc(statsDesc);
     stats.setStatsObj(colStatObjs);
 
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, -1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, null, -1);
 
     longStats.setNumDVs(40);
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, -1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, null, -1);
 
     List<String> colNames = new ArrayList<>();
     colNames.add(colName);
@@ -871,7 +871,7 @@ public class TestCachedStore {
     stats.setStatsDesc(statsDesc);
     stats.setStatsObj(colStatObjs);
 
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, -1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals1, null, -1);
 
     longStats.setNumDVs(40);
     hll = HyperLogLog.builder().build();
@@ -881,7 +881,7 @@ public class TestCachedStore {
     hll.addLong(5);
     longStats.setBitVectors(hll.serialize());
 
-    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, -1, null, -1);
+    cachedStore.updatePartitionColumnStatistics(stats.deepCopy(), partVals2, null, -1);
 
     List<String> colNames = new ArrayList<>();
     colNames.add(colName);