You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ct...@apache.org on 2017/05/01 13:58:40 UTC

[3/4] hive git commit: HIVE-16147: Rename a partitioned table should not drop its partition columns stats (Chaoyu Tang, reviewed by Pengcheng Xiong)

HIVE-16147: Rename a partitioned table should not drop its partition columns stats (Chaoyu Tang, reviewed by Pengcheng Xiong)


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

Branch: refs/heads/branch-2
Commit: 57de93a882ecb34d269ed1448229604a6abbb684
Parents: e29528c
Author: Chaoyu Tang <ct...@cloudera.com>
Authored: Mon May 1 09:39:38 2017 -0400
Committer: Chaoyu Tang <ct...@cloudera.com>
Committed: Mon May 1 09:54:58 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hive/metastore/HiveAlterHandler.java |  306 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |    8 -
 .../hadoop/hive/metastore/ObjectStore.java      |    7 +-
 .../clientpositive/alter_table_column_stats.q   |  241 ++
 .../alter_table_invalidate_column_stats.q       |  153 -
 .../alter_table_column_stats.q.out              | 2706 ++++++++++++++++++
 .../alter_table_invalidate_column_stats.q.out   |  932 ------
 7 files changed, 3120 insertions(+), 1233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/57de93a8/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index 15f2597..d281c03 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -54,8 +54,11 @@ import org.apache.hive.common.util.HiveStringUtils;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 
 /**
  * Hive specific implementation of alter
@@ -88,6 +91,9 @@ public class HiveAlterHandler implements AlterHandler {
   public void alterTable(RawStore msdb, Warehouse wh, String dbname,
       String name, Table newt, EnvironmentContext environmentContext,
       HMSHandler handler) throws InvalidOperationException, MetaException {
+    name = name.toLowerCase();
+    dbname = dbname.toLowerCase();
+
     final boolean cascade = environmentContext != null
         && environmentContext.isSetProperties()
         && StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(
@@ -96,9 +102,11 @@ public class HiveAlterHandler implements AlterHandler {
       throw new InvalidOperationException("New table is invalid: " + newt);
     }
 
-    if (!MetaStoreUtils.validateName(newt.getTableName(), hiveConf)) {
-      throw new InvalidOperationException(newt.getTableName()
-          + " is not a valid object name");
+    String newTblName = newt.getTableName().toLowerCase();
+    String newDbName = newt.getDbName().toLowerCase();
+
+    if (!MetaStoreUtils.validateName(newTblName, hiveConf)) {
+      throw new InvalidOperationException(newTblName + " is not a valid object name");
     }
     String validate = MetaStoreUtils.validateTblColumns(newt.getSd().getCols());
     if (validate != null) {
@@ -112,7 +120,6 @@ public class HiveAlterHandler implements AlterHandler {
 
     boolean success = false;
     boolean dataWasMoved = false;
-    boolean rename = false;
     Table oldt = null;
     List<MetaStoreEventListener> transactionalListeners = null;
     if (handler != null) {
@@ -120,26 +127,30 @@ public class HiveAlterHandler implements AlterHandler {
     }
 
     try {
-      msdb.openTransaction();
-      name = name.toLowerCase();
-      dbname = dbname.toLowerCase();
+      boolean rename = false;
+      boolean isPartitionedTable = false;
+      List<Partition> parts = null;
 
       // check if table with the new name already exists
-      if (!newt.getTableName().equalsIgnoreCase(name)
-          || !newt.getDbName().equalsIgnoreCase(dbname)) {
-        if (msdb.getTable(newt.getDbName(), newt.getTableName()) != null) {
-          throw new InvalidOperationException("new table " + newt.getDbName()
-              + "." + newt.getTableName() + " already exists");
+      if (!newTblName.equals(name) || !newDbName.equals(dbname)) {
+        if (msdb.getTable(newDbName, newTblName) != null) {
+          throw new InvalidOperationException("new table " + newDbName
+              + "." + newTblName + " already exists");
         }
         rename = true;
       }
 
+      msdb.openTransaction();
       // get old table
       oldt = msdb.getTable(dbname, name);
       if (oldt == null) {
         throw new InvalidOperationException("table " + dbname + "." + name + " doesn't exist");
       }
 
+      if (oldt.getPartitionKeysSize() != 0) {
+        isPartitionedTable = true;
+      }
+
       if (HiveConf.getBoolVar(hiveConf,
             HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES,
             false)) {
@@ -149,32 +160,14 @@ public class HiveAlterHandler implements AlterHandler {
             oldt.getSd().getCols(), newt.getSd().getCols());
       }
 
-      if (cascade) {
-        //Currently only column related changes can be cascaded in alter table
-        if(MetaStoreUtils.isCascadeNeededInAlterTable(oldt, newt)) {
-          List<Partition> parts = msdb.getPartitions(dbname, name, -1);
-          for (Partition part : parts) {
-            List<FieldSchema> oldCols = part.getSd().getCols();
-            part.getSd().setCols(newt.getSd().getCols());
-            String oldPartName = Warehouse.makePartName(oldt.getPartitionKeys(), part.getValues());
-            updatePartColumnStatsForAlterColumns(msdb, part, oldPartName, part.getValues(), oldCols, part);
-            msdb.alterPartition(dbname, name, part.getValues(), part);
-          }
-        } else {
-          LOG.warn("Alter table does not cascade changes to its partitions.");
-        }
-      }
-
       //check that partition keys have not changed, except for virtual views
       //however, allow the partition comments to change
       boolean partKeysPartiallyEqual = checkPartialPartKeysEqual(oldt.getPartitionKeys(),
           newt.getPartitionKeys());
 
       if(!oldt.getTableType().equals(TableType.VIRTUAL_VIEW.toString())){
-        if (oldt.getPartitionKeys().size() != newt.getPartitionKeys().size()
-            || !partKeysPartiallyEqual) {
-          throw new InvalidOperationException(
-              "partition keys can not be changed.");
+        if (!partKeysPartiallyEqual) {
+          throw new InvalidOperationException("partition keys can not be changed.");
         }
       }
 
@@ -205,9 +198,9 @@ public class HiveAlterHandler implements AlterHandler {
           srcFs = wh.getFs(srcPath);
 
           // get new location
-          Database db = msdb.getDatabase(newt.getDbName());
+          Database db = msdb.getDatabase(newDbName);
           Path databasePath = constructRenamedPath(wh.getDatabasePath(db), srcPath);
-          destPath = new Path(databasePath, newt.getTableName().toLowerCase());
+          destPath = new Path(databasePath, newTblName);
           destFs = wh.getFs(destPath);
 
           newt.getSd().setLocation(destPath.toString());
@@ -224,8 +217,7 @@ public class HiveAlterHandler implements AlterHandler {
           try {
             if (destFs.exists(destPath)) {
               throw new InvalidOperationException("New location for this table "
-                  + newt.getDbName() + "." + newt.getTableName()
-                  + " already exists : " + destPath);
+                  + newDbName + "." + newTblName + " already exists : " + destPath);
             }
             // check that src exists and also checks permissions necessary, rename src to dest
             if (srcFs.exists(srcPath) && srcFs.rename(srcPath, destPath)) {
@@ -237,38 +229,78 @@ public class HiveAlterHandler implements AlterHandler {
                 " failed to move data due to: '" + getSimpleMessage(e)
                 + "' See hive log file for details.");
           }
+        }
+
+        if (isPartitionedTable) {
           String oldTblLocPath = srcPath.toUri().getPath();
-          String newTblLocPath = destPath.toUri().getPath();
+          String newTblLocPath = dataWasMoved ? destPath.toUri().getPath() : null;
 
           // also the location field in partition
-          List<Partition> parts = msdb.getPartitions(dbname, name, -1);
+          parts = msdb.getPartitions(dbname, name, -1);
+          Map<Partition, ColumnStatistics> columnStatsNeedUpdated = new HashMap<Partition, ColumnStatistics>();
           for (Partition part : parts) {
             String oldPartLoc = part.getSd().getLocation();
-            if (oldPartLoc.contains(oldTblLocPath)) {
+            if (dataWasMoved && oldPartLoc.contains(oldTblLocPath)) {
               URI oldUri = new Path(oldPartLoc).toUri();
               String newPath = oldUri.getPath().replace(oldTblLocPath, newTblLocPath);
               Path newPartLocPath = new Path(oldUri.getScheme(), oldUri.getAuthority(), newPath);
               part.getSd().setLocation(newPartLocPath.toString());
-              String oldPartName = Warehouse.makePartName(oldt.getPartitionKeys(), part.getValues());
-              try {
-                //existing partition column stats is no longer valid, remove them
-                msdb.deletePartitionColumnStatistics(dbname, name, oldPartName, part.getValues(), null);
-              } catch (InvalidInputException iie) {
-                throw new InvalidOperationException("Unable to update partition stats in table rename." + iie);
-              }
-              msdb.alterPartition(dbname, name, part.getValues(), part);
             }
+            part.setDbName(newDbName);
+            part.setTableName(newTblName);
+            ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+                part.getValues(), part.getSd().getCols(), oldt, part);
+            if (colStats != null) {
+              columnStatsNeedUpdated.put(part, colStats);
+            }
+          }
+          msdb.alterTable(dbname, name, newt);
+          // alterPartition is only for changing the partition location in the table rename
+          if (dataWasMoved) {
+            for (Partition part : parts) {
+              msdb.alterPartition(newDbName, newTblName, part.getValues(), part);
+            }
+          }
+
+          for (Entry<Partition, ColumnStatistics> partColStats : columnStatsNeedUpdated.entrySet()) {
+            ColumnStatistics newPartColStats = partColStats.getValue();
+            newPartColStats.getStatsDesc().setDbName(newDbName);
+            newPartColStats.getStatsDesc().setTableName(newTblName);
+            msdb.updatePartitionColumnStatistics(newPartColStats, partColStats.getKey().getValues());
           }
+        } else {
+          alterTableUpdateTableColumnStats(msdb, oldt, newt);
         }
-      } else if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&
-        (newt.getPartitionKeysSize() == 0)) {
-          Database db = msdb.getDatabase(newt.getDbName());
-          // Update table stats. For partitioned table, we update stats in
-          // alterPartition()
+      } else {
+        // operations other than table rename
+        if (MetaStoreUtils.requireCalStats(hiveConf, null, null, newt, environmentContext) &&
+            !isPartitionedTable) {
+          Database db = msdb.getDatabase(newDbName);
+          // Update table stats. For partitioned table, we update stats in alterPartition()
           MetaStoreUtils.updateTableStatsFast(db, newt, wh, false, true, environmentContext);
+        }
+
+        if (cascade && isPartitionedTable) {
+          //Currently only column related changes can be cascaded in alter table
+          if(!MetaStoreUtils.areSameColumns(oldt.getSd().getCols(), newt.getSd().getCols())) {
+            parts = msdb.getPartitions(dbname, name, -1);
+            for (Partition part : parts) {
+              List<FieldSchema> oldCols = part.getSd().getCols();
+              part.getSd().setCols(newt.getSd().getCols());
+              ColumnStatistics colStats = updateOrGetPartitionColumnStats(msdb, dbname, name,
+                  part.getValues(), oldCols, oldt, part);
+              assert(colStats == null);
+              msdb.alterPartition(dbname, name, part.getValues(), part);
+            }
+            msdb.alterTable(dbname, name, newt);
+          } else {
+            LOG.warn("Alter table does not cascade changes to its partitions.");
+          }
+        } else {
+          alterTableUpdateTableColumnStats(msdb, oldt, newt);
+        }
       }
 
-      alterTableUpdateTableColumnStats(msdb, oldt, newt);
       if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
         MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
                                               EventMessage.EventType.ALTER_TABLE,
@@ -282,6 +314,11 @@ public class HiveAlterHandler implements AlterHandler {
       throw new InvalidOperationException(
           "Unable to change partition or table."
               + " Check metastore logs for detailed stack." + e.getMessage());
+    } catch (InvalidInputException e) {
+        LOG.debug("Accessing Metastore failed due to invalid input ", e);
+        throw new InvalidOperationException(
+            "Unable to change partition or table."
+                + " Check metastore logs for detailed stack." + e.getMessage());
     } catch (NoSuchObjectException e) {
       LOG.debug("Object not found in metastore ", e);
       throw new InvalidOperationException(
@@ -379,7 +416,11 @@ public class HiveAlterHandler implements AlterHandler {
           }
         }
 
-        updatePartColumnStats(msdb, dbname, name, new_part.getValues(), new_part);
+        // PartitionView does not have SD. We do not need update its column stats
+        if (oldPart.getSd() != null) {
+          updateOrGetPartitionColumnStats(msdb, dbname, name, new_part.getValues(),
+              oldPart.getSd().getCols(), tbl, new_part);
+        }
         msdb.alterPartition(dbname, name, new_part.getValues(), new_part);
         if (transactionalListeners != null && !transactionalListeners.isEmpty()) {
           MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
@@ -608,7 +649,12 @@ public class HiveAlterHandler implements AlterHandler {
             MetaStoreUtils.updatePartitionStatsFast(tmpPart, wh, false, true, environmentContext);
           }
         }
-        updatePartColumnStats(msdb, dbname, name, oldTmpPart.getValues(), tmpPart);
+
+        // PartitionView does not have SD and we do not need to update its column stats
+        if (oldTmpPart.getSd() != null) {
+          updateOrGetPartitionColumnStats(msdb, dbname, name, oldTmpPart.getValues(),
+              oldTmpPart.getSd().getCols(), tbl, tmpPart);
+        }
       }
 
       msdb.alterPartitions(dbname, name, partValsList, new_parts);
@@ -678,91 +724,8 @@ public class HiveAlterHandler implements AlterHandler {
         defaultNewPath.toUri().getPath());
   }
 
-  private void updatePartColumnStatsForAlterColumns(RawStore msdb, Partition oldPartition,
-      String oldPartName, List<String> partVals, List<FieldSchema> oldCols, Partition newPart)
-          throws MetaException, InvalidObjectException {
-    String dbName = oldPartition.getDbName();
-    String tableName = oldPartition.getTableName();
-    try {
-      List<String> oldPartNames = Lists.newArrayList(oldPartName);
-      List<String> oldColNames = new ArrayList<String>(oldCols.size());
-      for (FieldSchema oldCol : oldCols) {
-        oldColNames.add(oldCol.getName());
-      }
-      List<FieldSchema> newCols = newPart.getSd().getCols();
-      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(dbName, tableName,
-          oldPartNames, oldColNames);
-      assert (partsColStats.size() <= 1);
-      for (ColumnStatistics partColStats : partsColStats) { //actually only at most one loop
-        List<ColumnStatisticsObj> statsObjs = partColStats.getStatsObj();
-        List<String> deletedCols = new ArrayList<String>();
-        for (ColumnStatisticsObj statsObj : statsObjs) {
-          boolean found =false;
-          for (FieldSchema newCol : newCols) {
-            if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
-                && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
-              found = true;
-              break;
-            }
-          }
-          if (!found) {
-            msdb.deletePartitionColumnStatistics(dbName, tableName, oldPartName, partVals,
-                statsObj.getColName());
-            deletedCols.add(statsObj.getColName());
-          }
-        }
-        StatsSetupConst.removeColumnStatsState(newPart.getParameters(), deletedCols);
-      }
-    } catch (NoSuchObjectException nsoe) {
-      LOG.debug("Could not find db entry." + nsoe);
-      //ignore
-    } catch (InvalidInputException iie) {
-      throw new InvalidObjectException
-      ("Invalid input to update partition column stats in alter table change columns" + iie);
-    }
-  }
-
-  private void updatePartColumnStats(RawStore msdb, String dbName, String tableName,
-      List<String> partVals, Partition newPart) throws MetaException, InvalidObjectException {
-    dbName = HiveStringUtils.normalizeIdentifier(dbName);
-    tableName = HiveStringUtils.normalizeIdentifier(tableName);
-    String newDbName = HiveStringUtils.normalizeIdentifier(newPart.getDbName());
-    String newTableName = HiveStringUtils.normalizeIdentifier(newPart.getTableName());
-
-    Table oldTable = msdb.getTable(dbName, tableName);
-    if (oldTable == null) {
-      return;
-    }
-
-    try {
-      String oldPartName = Warehouse.makePartName(oldTable.getPartitionKeys(), partVals);
-      String newPartName = Warehouse.makePartName(oldTable.getPartitionKeys(), newPart.getValues());
-      if (!dbName.equals(newDbName) || !tableName.equals(newTableName)
-          || !oldPartName.equals(newPartName)) {
-        msdb.deletePartitionColumnStatistics(dbName, tableName, oldPartName, partVals, null);
-      } else {
-        Partition oldPartition = msdb.getPartition(dbName, tableName, partVals);
-        if (oldPartition == null) {
-          return;
-        }
-        if (oldPartition.getSd() != null && newPart.getSd() != null) {
-        List<FieldSchema> oldCols = oldPartition.getSd().getCols();
-          if (!MetaStoreUtils.columnsIncluded(oldCols, newPart.getSd().getCols())) {
-            updatePartColumnStatsForAlterColumns(msdb, oldPartition, oldPartName, partVals, oldCols, newPart);
-          }
-        }
-      }
-    } catch (NoSuchObjectException nsoe) {
-      LOG.debug("Could not find db entry." + nsoe);
-      //ignore
-    } catch (InvalidInputException iie) {
-      throw new InvalidObjectException("Invalid input to update partition column stats." + iie);
-    }
-  }
-
   @VisibleForTesting
-  void alterTableUpdateTableColumnStats(RawStore msdb,
-      Table oldTable, Table newTable)
+  void alterTableUpdateTableColumnStats(RawStore msdb, Table oldTable, Table newTable)
       throws MetaException, InvalidObjectException {
     String dbName = oldTable.getDbName().toLowerCase();
     String tableName = HiveStringUtils.normalizeIdentifier(oldTable.getTableName());
@@ -839,4 +802,69 @@ public class HiveAlterHandler implements AlterHandler {
       throw new InvalidObjectException("Invalid inputs to update table column stats: " + e);
     }
   }
+
+  private ColumnStatistics updateOrGetPartitionColumnStats(
+      RawStore msdb, String dbname, String tblname, List<String> partVals,
+      List<FieldSchema> oldCols, Table table, Partition part)
+          throws MetaException, InvalidObjectException {
+    ColumnStatistics newPartsColStats = null;
+    try {
+      List<FieldSchema> newCols = part.getSd() == null ?
+          new ArrayList<FieldSchema>() : part.getSd().getCols();
+      String oldPartName = Warehouse.makePartName(table.getPartitionKeys(), partVals);
+      String newPartName = Warehouse.makePartName(table.getPartitionKeys(), part.getValues());
+      boolean rename = !part.getDbName().equals(dbname) || !part.getTableName().equals(tblname)
+          || !oldPartName.equals(newPartName);
+
+      // do not need to update column stats if alter partition is not for rename or changing existing columns
+      if (!rename && MetaStoreUtils.columnsIncluded(oldCols, newCols)) {
+        return newPartsColStats;
+      }
+      List<String> oldColNames = new ArrayList<String>(oldCols.size());
+      for (FieldSchema oldCol : oldCols) {
+        oldColNames.add(oldCol.getName());
+      }
+      List<String> oldPartNames = Lists.newArrayList(oldPartName);
+      List<ColumnStatistics> partsColStats = msdb.getPartitionColumnStatistics(dbname, tblname,
+          oldPartNames, oldColNames);
+      assert (partsColStats.size() <= 1);
+      for (ColumnStatistics partColStats : partsColStats) { //actually only at most one loop
+        List<ColumnStatisticsObj> newStatsObjs = new ArrayList<ColumnStatisticsObj>();
+        List<ColumnStatisticsObj> statsObjs = partColStats.getStatsObj();
+        List<String> deletedCols = new ArrayList<String>();
+        for (ColumnStatisticsObj statsObj : statsObjs) {
+          boolean found =false;
+          for (FieldSchema newCol : newCols) {
+            if (statsObj.getColName().equalsIgnoreCase(newCol.getName())
+                && statsObj.getColType().equalsIgnoreCase(newCol.getType())) {
+              found = true;
+              break;
+            }
+          }
+          if (found) {
+            if (rename) {
+              msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+                  partVals, statsObj.getColName());
+              newStatsObjs.add(statsObj);
+            }
+          } else {
+            msdb.deletePartitionColumnStatistics(dbname, tblname, partColStats.getStatsDesc().getPartName(),
+                partVals, statsObj.getColName());
+            deletedCols.add(statsObj.getColName());
+          }
+        }
+        StatsSetupConst.removeColumnStatsState(part.getParameters(), deletedCols);
+        if (!newStatsObjs.isEmpty()) {
+          partColStats.setStatsObj(newStatsObjs);
+          newPartsColStats = partColStats;
+        }
+      }
+    } catch (NoSuchObjectException nsoe) {
+      // ignore this exception, actually this exception won't be thrown from getPartitionColumnStatistics
+    } catch (InvalidInputException iie) {
+      throw new InvalidObjectException("Invalid input to delete partition column stats." + iie);
+    }
+
+    return newPartsColStats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57de93a8/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index 79f6d7f..5418dd6 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -635,14 +635,6 @@ public class MetaStoreUtils {
     }
   }
 
-  static boolean isCascadeNeededInAlterTable(Table oldTable, Table newTable) {
-    //currently cascade only supports add/replace columns and
-    //changing column type/position/name/comments
-    List<FieldSchema> oldCols = oldTable.getSd().getCols();
-    List<FieldSchema> newCols = newTable.getSd().getCols();
-    return !areSameColumns(oldCols, newCols);
-  }
-
   static boolean areSameColumns(List<FieldSchema> oldCols, List<FieldSchema> newCols) {
     if (oldCols.size() != newCols.size()) {
       return false;

http://git-wip-us.apache.org/repos/asf/hive/blob/57de93a8/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 358cf17..ae187cf 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -7379,7 +7379,12 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       // We are not going to verify SD for each partition. Just verify for the table.
-      validateTableCols(table, colNames);
+      // ToDo: we need verify the partition column instead
+      try {
+        validateTableCols(table, colNames);
+      } catch (MetaException me) {
+        LOG.warn("The table does not have the same column definition as its partition.");
+      }
       Query query = queryWrapper.query = pm.newQuery(MPartitionColumnStatistics.class);
       String paramStr = "java.lang.String t1, java.lang.String t2";
       String filter = "tableName == t1 && dbName == t2 && (";

http://git-wip-us.apache.org/repos/asf/hive/blob/57de93a8/ql/src/test/queries/clientpositive/alter_table_column_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_table_column_stats.q b/ql/src/test/queries/clientpositive/alter_table_column_stats.q
new file mode 100644
index 0000000..39dfb0c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/alter_table_column_stats.q
@@ -0,0 +1,241 @@
+set hive.mapred.mode=nonstrict;
+
+set hive.metastore.try.direct.sql=true;
+
+drop database if exists statsdb1;
+create database statsdb1;
+drop database if exists statsdb2;
+create database statsdb2;
+
+create table statsdb1.testtable0 (col1 int, col2 string, col3 string);
+insert into statsdb1.testtable0 select key, value, 'val3' from src limit 10;
+
+create table statsdb1.testpart0 (col1 int, col2 string, col3 string) partitioned by (part string);
+insert into statsdb1.testpart0 partition (part = 'part1') select key, value, 'val3' from src limit 10;
+insert into statsdb1.testpart0 partition (part = 'part2') select key, value, 'val3' from src limit 20;
+
+use statsdb1;
+-- test non-partitioned table
+analyze table testtable0 compute statistics for columns;
+describe formatted statsdb1.testtable0;
+describe formatted statsdb1.testtable0 col1;
+describe formatted statsdb1.testtable0 col2;
+describe formatted statsdb1.testtable0 col3;
+
+-- rename non-partitioned table should not change its table and columns stats
+alter table statsdb1.testtable0 rename to statsdb1.testtable1;
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col3;
+
+-- when replacing columns in a non-partitioned table, the table stats should not change,
+-- but the stats of the changed columns are removed
+alter table testtable1 replace columns (col1 int, col2 string, col4 string);
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col4;
+
+-- when changing the column type in a non-partitioned table, the table stats should not change,
+-- but the stats of the type-changed columns are removed
+alter table testtable1 change col1 col1 string;
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col4;
+
+-- rename the db of a non-partitoned table, the table and columns stats should not change
+alter table statsdb1.testtable1 rename to statsdb2.testtable2;
+describe formatted statsdb2.testtable2;
+describe formatted statsdb2.testtable2 col1;
+describe formatted statsdb2.testtable2 col2;
+describe formatted statsdb2.testtable2 col4;
+
+-- test partitioned table
+analyze table testpart0 compute statistics for columns;
+describe formatted statsdb1.testpart0;
+describe formatted statsdb1.testpart0 partition (part = 'part1');
+describe formatted statsdb1.testpart0 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart0 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart0 partition (part = 'part1') col3;
+describe formatted statsdb1.testpart0 partition (part = 'part2');
+describe formatted statsdb1.testpart0 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart0 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart0 partition (part = 'part2') col3;
+
+-- rename a partitioned table should not change its table, partition, and column stats
+alter table statsdb1.testpart0 rename to statsdb1.testpart1;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col3;
+
+-- when cascade replacing columns in a partitioned table, the table and partition stats should not change,
+-- but the stats of the changed columns are removed
+alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col4;
+
+-- when cascade changing the column type in a partitioned table, the table and partition stats should not change,
+-- but the stats of the type-changed columns are removed
+alter table statsdb1.testpart1 change column col1 col1 string cascade;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col4;
+
+-- change database of a partition should not change table, partition and columns stats
+alter table statsdb1.testpart1 rename to statsdb2.testpart2;
+describe formatted statsdb2.testpart2;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col1;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col2;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col4;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col1;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col2;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col4;
+
+use statsdb2;
+drop table statsdb2.testpart2;
+drop table statsdb2.testtable2;
+
+use default;
+drop database statsdb1;
+drop database statsdb2;
+
+set hive.metastore.try.direct.sql=false;
+
+drop database if exists statsdb1;
+create database statsdb1;
+drop database if exists statsdb2;
+create database statsdb2;
+
+create table statsdb1.testtable0 (col1 int, col2 string, col3 string);
+insert into statsdb1.testtable0 select key, value, 'val3' from src limit 10;
+
+create table statsdb1.testpart0 (col1 int, col2 string, col3 string) partitioned by (part string);
+insert into statsdb1.testpart0 partition (part = 'part1') select key, value, 'val3' from src limit 10;
+insert into statsdb1.testpart0 partition (part = 'part2') select key, value, 'val3' from src limit 20;
+
+use statsdb1;
+-- test non-partitioned table
+analyze table testtable0 compute statistics for columns;
+describe formatted statsdb1.testtable0;
+describe formatted statsdb1.testtable0 col1;
+describe formatted statsdb1.testtable0 col2;
+describe formatted statsdb1.testtable0 col3;
+
+-- rename non-partitioned table should not change its table and columns stats
+alter table statsdb1.testtable0 rename to statsdb1.testtable1;
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col3;
+
+-- when replacing columns in a non-partitioned table, the table stats should not change,
+-- but the stats of the changed columns are removed
+alter table testtable1 replace columns (col1 int, col2 string, col4 string);
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col4;
+
+-- when changing the column type in a non-partitioned table, the table stats should not change,
+-- but the stats of the type-changed columns are removed
+alter table testtable1 change col1 col1 string;
+describe formatted statsdb1.testtable1;
+describe formatted statsdb1.testtable1 col1;
+describe formatted statsdb1.testtable1 col2;
+describe formatted statsdb1.testtable1 col4;
+
+-- rename the db of a non-partitoned table, the table and columns stats should not change
+alter table statsdb1.testtable1 rename to statsdb2.testtable2;
+describe formatted statsdb2.testtable2;
+describe formatted statsdb2.testtable2 col1;
+describe formatted statsdb2.testtable2 col2;
+describe formatted statsdb2.testtable2 col4;
+
+-- test partitioned table
+analyze table testpart0 compute statistics for columns;
+describe formatted statsdb1.testpart0;
+describe formatted statsdb1.testpart0 partition (part = 'part1');
+describe formatted statsdb1.testpart0 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart0 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart0 partition (part = 'part1') col3;
+describe formatted statsdb1.testpart0 partition (part = 'part2');
+describe formatted statsdb1.testpart0 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart0 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart0 partition (part = 'part2') col3;
+
+-- rename a partitioned table should not change its table, partition, and column stats
+alter table statsdb1.testpart0 rename to statsdb1.testpart1;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col3;
+
+-- when cascade replacing columns in a partitioned table, the table and partition stats should not change,
+-- but the stats of the changed columns are removed
+alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col4;
+
+-- when cascade changing the column type in a partitioned table, the table and partition stats should not change,
+-- but the stats of the type-changed columns are removed
+alter table statsdb1.testpart1 change column col1 col1 string cascade;
+describe formatted statsdb1.testpart1;
+describe formatted statsdb1.testpart1 partition (part = 'part1');
+describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
+describe formatted statsdb1.testpart1 partition (part = 'part2');
+describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
+describe formatted statsdb1.testpart1 partition (part = 'part2') col4;
+
+-- change database of a partition should not change table, partition and columns stats
+alter table statsdb1.testpart1 rename to statsdb2.testpart2;
+describe formatted statsdb2.testpart2;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col1;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col2;
+describe formatted statsdb2.testpart2 partition (part = 'part1') col4;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col1;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col2;
+describe formatted statsdb2.testpart2 partition (part = 'part2') col4;
+
+use statsdb2;
+drop table statsdb2.testpart2;
+drop table statsdb2.testtable2;
+
+use default;
+drop database statsdb1;
+drop database statsdb2;

http://git-wip-us.apache.org/repos/asf/hive/blob/57de93a8/ql/src/test/queries/clientpositive/alter_table_invalidate_column_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_table_invalidate_column_stats.q b/ql/src/test/queries/clientpositive/alter_table_invalidate_column_stats.q
deleted file mode 100644
index a478451..0000000
--- a/ql/src/test/queries/clientpositive/alter_table_invalidate_column_stats.q
+++ /dev/null
@@ -1,153 +0,0 @@
-set hive.mapred.mode=nonstrict;
-set hive.metastore.try.direct.sql=true;
-
-drop database if exists statsdb1;
-create database statsdb1;
-drop database if exists statsdb2;
-create database statsdb2;
-
-create table statsdb1.testtable1 (col1 int, col2 string, col3 string);
-insert into statsdb1.testtable1 select key, value, 'val3' from src limit 10;
-
-create table statsdb1.testpart1 (col1 int, col2 string, col3 string) partitioned by (part string);
-insert into statsdb1.testpart1 partition (part = 'part1') select key, value, 'val3' from src limit 10;
-insert into statsdb1.testpart1 partition (part = 'part2') select key, value, 'val3' from src limit 20;
-
-use statsdb1;
-
-analyze table testtable1 compute statistics for columns;
-
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col3;
-
-alter table testtable1 replace columns (col1 int, col2 string, col4 string);
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col4;
-
-alter table testtable1 change col1 col1 string;
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col4;
-
-alter table statsdb1.testtable1 rename to statsdb2.testtable2;
-
-
-analyze table testpart1 compute statistics for columns;
-
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col3;
-
-alter table statsdb1.testpart1 partition (part = 'part2') rename to partition (part = 'part3');
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col3;
-
-alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
-
-alter table statsdb1.testpart1 change column col1 col1 string;
-set hive.exec.dynamic.partition = true;
-alter table statsdb1.testpart1 partition (part) change column col1 col1 string;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
-
-alter table statsdb1.testpart1 rename to statsdb2.testpart2;
-use statsdb2;
-
-alter table statsdb2.testpart2 drop partition (part = 'part1');
-drop table statsdb2.testpart2;
-
-drop table statsdb2.testtable2;
-
-use default;
-drop database statsdb1;
-drop database statsdb2;
-
-
-set hive.metastore.try.direct.sql=false;
-
-drop database if exists statsdb1;
-create database statsdb1;
-drop database if exists statsdb2;
-create database statsdb2;
-
-create table statsdb1.testtable1 (col1 int, col2 string, col3 string);
-insert into statsdb1.testtable1 select key, value, 'val3' from src limit 10;
-
-create table statsdb1.testpart1 (col1 int, col2 string, col3 string) partitioned by (part string);
-insert into statsdb1.testpart1 partition (part = 'part1') select key, value, 'val3' from src limit 10;
-insert into statsdb1.testpart1 partition (part = 'part2') select key, value, 'val3' from src limit 20;
-
-use statsdb1;
-
-analyze table testtable1 compute statistics for columns;
-
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col3;
-
-alter table testtable1 replace columns (col1 int, col2 string, col4 string);
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col4;
-
-alter table testtable1 change col1 col1 string;
-describe formatted statsdb1.testtable1 col1;
-describe formatted statsdb1.testtable1 col2;
-describe formatted statsdb1.testtable1 col4;
-
-alter table statsdb1.testtable1 rename to statsdb2.testtable2;
-
-
-analyze table testpart1 compute statistics for columns;
-
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part2') col3;
-
-alter table statsdb1.testpart1 partition (part = 'part2') rename to partition (part = 'part3');
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col3;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part3') col3;
-
-alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
-
-alter table statsdb1.testpart1 change column col1 col1 string;
-set hive.exec.dynamic.partition = true;
-alter table statsdb1.testpart1 partition (part) change column col1 col1 string;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col1;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col2;
-describe formatted statsdb1.testpart1 partition (part = 'part1') col4;
-
-alter table statsdb1.testpart1 rename to statsdb2.testpart2;
-use statsdb2;
-
-alter table statsdb2.testpart2 drop partition (part = 'part1');
-drop table statsdb2.testpart2;
-
-drop table statsdb2.testtable2;
-
-use default;
-drop database statsdb1;
-drop database statsdb2;