You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by de...@apache.org on 2022/03/02 01:43:44 UTC

[hive] branch master updated: HIVE-24949: Fail to rename a partition with customized catalog (#2910) (Zhihua Deng reviewed by Zoltan Haindrich)

This is an automated email from the ASF dual-hosted git repository.

dengzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new f15e67b  HIVE-24949: Fail to rename a partition with customized catalog (#2910) (Zhihua Deng reviewed by Zoltan Haindrich)
f15e67b is described below

commit f15e67b8ff119b41e3257ce6ef5864cece0eaf9b
Author: dengzh <de...@gmail.com>
AuthorDate: Wed Mar 2 09:43:29 2022 +0800

    HIVE-24949: Fail to rename a partition with customized catalog (#2910) (Zhihua Deng reviewed by Zoltan Haindrich)
---
 .../hcatalog/listener/DbNotificationListener.java  | 44 +++++++++++-----------
 .../ql/metadata/SessionHiveMetaStoreClient.java    |  3 +-
 .../hadoop/hive/metastore/DirectSqlUpdateStat.java |  8 +++-
 .../apache/hadoop/hive/metastore/HMSHandler.java   | 10 ++---
 .../hadoop/hive/metastore/HiveAlterHandler.java    |  7 ++--
 .../apache/hadoop/hive/metastore/ObjectStore.java  |  3 +-
 .../hadoop/hive/metastore/StatObjectConverter.java | 13 ++++---
 .../hadoop/hive/metastore/cache/CachedStore.java   |  7 ++--
 .../hive/metastore/VerifyingObjectStore.java       |  1 -
 9 files changed, 49 insertions(+), 47 deletions(-)

diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 7980d53..fba6827 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -141,8 +141,8 @@ import org.apache.hive.hcatalog.data.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import com.google.common.collect.Lists;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.EVENT_DB_LISTENER_CLEAN_STARTUP_WAIT_INTERVAL;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 
 /**
  * An implementation of {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} that
@@ -278,7 +278,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_TABLE.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(t.isSetCatName() ? t.getCatName() : getDefaultCatalog(conf));
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, tableEvent);
@@ -295,7 +295,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_TABLE.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(t.isSetCatName() ? t.getCatName() : getDefaultCatalog(conf));
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, tableEvent);
@@ -316,7 +316,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         new NotificationEvent(0, now(), EventType.ALTER_TABLE.toString(),
             msgEncoder.getSerializer().serialize(msg)
         );
-    event.setCatName(after.isSetCatName() ? after.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(after.isSetCatName() ? after.getCatName() : getDefaultCatalog(conf));
     event.setDbName(after.getDbName());
     event.setTableName(after.getTableName());
     process(event, tableEvent);
@@ -434,7 +434,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
 
     NotificationEvent event = new NotificationEvent(0, now(),
         EventType.ADD_PARTITION.toString(), serializer.serialize(msg));
-    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(t.isSetCatName() ? t.getCatName() : getDefaultCatalog(conf));
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, partitionEvent);
@@ -452,7 +452,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
             .buildDropPartitionMessage(t, partitionEvent.getPartitionIterator());
     NotificationEvent event = new NotificationEvent(0, now(), EventType.DROP_PARTITION.toString(),
         msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(t.isSetCatName() ? t.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(t.isSetCatName() ? t.getCatName() : getDefaultCatalog(conf));
     event.setDbName(t.getDbName());
     event.setTableName(t.getTableName());
     process(event, partitionEvent);
@@ -473,7 +473,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.ALTER_PARTITION.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(before.isSetCatName() ? before.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(before.isSetCatName() ? before.getCatName() : getDefaultCatalog(conf));
     event.setDbName(before.getDbName());
     event.setTableName(before.getTableName());
     process(event, partitionEvent);
@@ -491,7 +491,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_DATABASE.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : getDefaultCatalog(conf));
     event.setDbName(db.getName());
     process(event, dbEvent);
   }
@@ -508,7 +508,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_DATABASE.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(db.isSetCatalogName() ? db.getCatalogName() : getDefaultCatalog(conf));
     event.setDbName(db.getName());
     process(event, dbEvent);
   }
@@ -527,7 +527,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         new NotificationEvent(0, now(), EventType.ALTER_DATABASE.toString(),
             msgEncoder.getSerializer().serialize(msg)
         );
-    event.setCatName(oldDb.isSetCatalogName() ? oldDb.getCatalogName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(oldDb.isSetCatalogName() ? oldDb.getCatalogName() : getDefaultCatalog(conf));
     event.setDbName(oldDb.getName());
     process(event, dbEvent);
   }
@@ -544,7 +544,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.CREATE_FUNCTION.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(fn.isSetCatName() ? fn.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(fn.isSetCatName() ? fn.getCatName() : getDefaultCatalog(conf));
     event.setDbName(fn.getDbName());
     process(event, fnEvent);
   }
@@ -560,7 +560,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.DROP_FUNCTION.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(fn.isSetCatName() ? fn.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(fn.isSetCatName() ? fn.getCatName() : getDefaultCatalog(conf));
     event.setDbName(fn.getDbName());
     process(event, fnEvent);
   }
@@ -613,7 +613,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event =
         new NotificationEvent(0, now(), EventType.INSERT.toString(),
             msgEncoder.getSerializer().serialize(msg));
-    event.setCatName(tableObj.isSetCatName() ? tableObj.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(tableObj.isSetCatName() ? tableObj.getCatName() : getDefaultCatalog(conf));
     event.setDbName(tableObj.getDbName());
     event.setTableName(tableObj.getTableName());
     process(event, insertEvent);
@@ -700,7 +700,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
           .buildAddPrimaryKeyMessage(addPrimaryKeyEvent.getPrimaryKeyCols());
       NotificationEvent event = new NotificationEvent(0, now(), EventType.ADD_PRIMARYKEY.toString(),
           msgEncoder.getSerializer().serialize(msg));
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addPrimaryKeyEvent);
@@ -720,7 +720,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
           new NotificationEvent(0, now(), EventType.ADD_FOREIGNKEY.toString(),
               msgEncoder.getSerializer().serialize(msg));
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getPktable_db());
       event.setTableName(cols.get(0).getPktable_name());
       process(event, addForeignKeyEvent);
@@ -741,7 +741,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
           new NotificationEvent(0, now(), EventType.ADD_UNIQUECONSTRAINT.toString(),
               msgEncoder.getSerializer().serialize(msg)
           );
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addUniqueConstraintEvent);
@@ -762,7 +762,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
           new NotificationEvent(0, now(), EventType.ADD_NOTNULLCONSTRAINT.toString(),
               msgEncoder.getSerializer().serialize(msg)
           );
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addNotNullConstraintEvent);
@@ -783,7 +783,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         new NotificationEvent(0, now(), EventType.ADD_DEFAULTCONSTRAINT.toString(),
           msgEncoder.getSerializer().serialize(colsInMsg)
         );
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addDefaultConstraintEvent);
@@ -805,7 +805,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         new NotificationEvent(0, now(), EventType.ADD_CHECKCONSTRAINT.toString(),
           msgEncoder.getSerializer().serialize(colsInMsg)
         );
-      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(cols.get(0).isSetCatName() ? cols.get(0).getCatName() : getDefaultCatalog(conf));
       event.setDbName(cols.get(0).getTable_db());
       event.setTableName(cols.get(0).getTable_name());
       process(event, addCheckConstraintEvent);
@@ -914,7 +914,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event = new NotificationEvent(0, now(), EventType.UPDATE_TABLE_COLUMN_STAT.toString(),
                     msgEncoder.getSerializer().serialize(msg));
     ColumnStatisticsDesc statDesc = updateTableColumnStatEvent.getColStats().getStatsDesc();
-    event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : getDefaultCatalog(conf));
     event.setDbName(statDesc.getDbName());
     event.setTableName(statDesc.getTableName());
     process(event, updateTableColumnStatEvent);
@@ -944,7 +944,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     NotificationEvent event = new NotificationEvent(0, now(), EventType.UPDATE_PARTITION_COLUMN_STAT.toString(),
                     msgEncoder.getSerializer().serialize(msg));
     ColumnStatisticsDesc statDesc = updatePartColStatEvent.getPartColStats().getStatsDesc();
-    event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : DEFAULT_CATALOG_NAME);
+    event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : getDefaultCatalog(conf));
     event.setDbName(statDesc.getDbName());
     event.setTableName(statDesc.getTableName());
     process(event, updatePartColStatEvent);
@@ -969,7 +969,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       NotificationEvent event =
               new NotificationEvent(0, now(), EventType.UPDATE_PARTITION_COLUMN_STAT.toString(),
                       msgEncoder.getSerializer().serialize(msg));
-      event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : DEFAULT_CATALOG_NAME);
+      event.setCatName(statDesc.isSetCatName() ? statDesc.getCatName() : getDefaultCatalog(conf));
       event.setDbName(statDesc.getDbName());
       event.setTableName(statDesc.getTableName());
       eventBatch.add(event);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 4778c2e..38047fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -126,7 +126,6 @@ import static org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedTabl
 import static org.apache.hadoop.hive.metastore.Warehouse.makePartName;
 import static org.apache.hadoop.hive.metastore.Warehouse.makeSpecFromName;
 import static org.apache.hadoop.hive.metastore.Warehouse.makeValsFromName;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.compareFieldColumns;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getColumnNamesForTable;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
@@ -273,7 +272,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClientWithLocalCach
     GetTableRequest getTableRequest = new GetTableRequest(dbName, tableName);
     getTableRequest.setGetColumnStats(getColStats);
     getTableRequest.setEngine(engine);
-    if (!DEFAULT_CATALOG_NAME.equals(catName)) {
+    if (!getDefaultCatalog(conf).equals(catName)) {
       getTableRequest.setCatName(catName);
       return super.getTable(getTableRequest);
     } else {
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java
index 2fb9d20..6bd1c3d 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/DirectSqlUpdateStat.java
@@ -55,6 +55,7 @@ import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE;
 import static org.apache.hadoop.hive.metastore.HMSHandler.getPartValsFromName;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 
 /**
  * This class contains the optimizations for MetaStore that rely on direct SQL access to
@@ -148,7 +149,7 @@ class DirectSqlUpdateStat {
   private void populateInsertUpdateMap(Map<PartitionInfo, ColumnStatistics> statsPartInfoMap,
                                        Map<PartColNameInfo, MPartitionColumnStatistics> updateMap,
                                        Map<PartColNameInfo, MPartitionColumnStatistics>insertMap,
-                                       Connection dbConn) throws SQLException, MetaException, NoSuchObjectException {
+                                       Connection dbConn, Table tbl) throws SQLException, MetaException, NoSuchObjectException {
     StringBuilder prefix = new StringBuilder();
     StringBuilder suffix = new StringBuilder();
     Statement statement = null;
@@ -182,6 +183,9 @@ class DirectSqlUpdateStat {
       ColumnStatistics colStats = (ColumnStatistics) entry.getValue();
       long partId = partitionInfo.partitionId;
       ColumnStatisticsDesc statsDesc = colStats.getStatsDesc();
+      if (!statsDesc.isSetCatName()) {
+        statsDesc.setCatName(tbl.getCatName());
+      }
       for (ColumnStatisticsObj statisticsObj : colStats.getStatsObj()) {
         PartColNameInfo temp = new PartColNameInfo(partId, statisticsObj.getColName());
         if (selectedParts.contains(temp)) {
@@ -599,7 +603,7 @@ class DirectSqlUpdateStat {
 
       Map<PartColNameInfo, MPartitionColumnStatistics> insertMap = new HashMap<>();
       Map<PartColNameInfo, MPartitionColumnStatistics> updateMap = new HashMap<>();
-      populateInsertUpdateMap(partitionInfoMap, updateMap, insertMap, dbConn);
+      populateInsertUpdateMap(partitionInfoMap, updateMap, insertMap, dbConn, tbl);
 
       LOG.info("Number of stats to insert  " + insertMap.size() + " update " + updateMap.size());
 
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
index fc6d963..b2bd19a 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
@@ -5955,7 +5955,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
       environmentContext = new EnvironmentContext();
     }
     if (catName == null) {
-      catName = MetaStoreUtils.getDefaultCatalog(conf);
+      catName = getDefaultCatalog(conf);
     }
 
     startTableFunction("alter_partitions", catName, db_name, tbl_name);
@@ -6074,7 +6074,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
       envContext = new EnvironmentContext();
     }
     if (catName == null) {
-      catName = MetaStoreUtils.getDefaultCatalog(conf);
+      catName = getDefaultCatalog(conf);
     }
 
     // HIVE-25282: Drop/Alter table in REMOTE db should fail
@@ -9551,7 +9551,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
     ms.openTransaction();
     boolean success = false;
     try {
-      Table tbl = ms.getTable(DEFAULT_CATALOG_NAME, dbName, tblName);
+      Table tbl = ms.getTable(getDefaultCatalog(conf), dbName, tblName);
       if (tbl == null) {
         throw new NoSuchObjectException(dbName + "." + tblName + " not found");
       }
@@ -9576,7 +9576,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
         if (partName != null) {
           partNames = Lists.newArrayList(partName);
         } else if (isAllPart) {
-          partNames = ms.listPartitionNames(DEFAULT_CATALOG_NAME, dbName, tblName, (short)-1);
+          partNames = ms.listPartitionNames(getDefaultCatalog(conf), dbName, tblName, (short)-1);
         } else {
           throw new MetaException("Table is partitioned");
         }
@@ -9589,7 +9589,7 @@ public class HMSHandler extends FacebookBase implements IHMSHandler {
           int currentBatchSize = Math.min(batchSize, partNames.size() - index);
           List<String> nameBatch = partNames.subList(index, index + currentBatchSize);
           index += currentBatchSize;
-          List<Partition> parts = ms.getPartitionsByNames(DEFAULT_CATALOG_NAME, dbName, tblName, nameBatch);
+          List<Partition> parts = ms.getPartitionsByNames(getDefaultCatalog(conf), dbName, tblName, nameBatch);
           for (Partition part : parts) {
             if (!part.isSetSd() || !part.getSd().isSetLocation()) {
               throw new MetaException("Partition does not have storage location;" +
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
index 06b8d1f..283ea5b 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
@@ -69,7 +69,6 @@ import java.util.LinkedList;
 
 import static org.apache.hadoop.hive.metastore.HiveMetaHook.ALTERLOCATION;
 import static org.apache.hadoop.hive.metastore.HiveMetaHook.ALTER_TABLE_OPERATION_TYPE;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
 
@@ -512,7 +511,7 @@ public class HiveAlterHandler implements AlterHandler {
     final String name, final List<String> part_vals, final Partition new_part,
     EnvironmentContext environmentContext)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartition(msdb, wh, DEFAULT_CATALOG_NAME, dbname, name, part_vals, new_part,
+    return alterPartition(msdb, wh, MetaStoreUtils.getDefaultCatalog(conf), dbname, name, part_vals, new_part,
         environmentContext, null, null);
   }
 
@@ -600,7 +599,7 @@ public class HiveAlterHandler implements AlterHandler {
     Database db;
     try {
       msdb.openTransaction();
-      Table tbl = msdb.getTable(DEFAULT_CATALOG_NAME, dbname, name, null);
+      Table tbl = msdb.getTable(catName, dbname, name, null);
       if (tbl == null) {
         throw new InvalidObjectException(
             "Unable to alter partition because table or database does not exist.");
@@ -754,7 +753,7 @@ public class HiveAlterHandler implements AlterHandler {
     final String name, final List<Partition> new_parts,
     EnvironmentContext environmentContext)
       throws InvalidOperationException, InvalidObjectException, AlreadyExistsException, MetaException {
-    return alterPartitions(msdb, wh, DEFAULT_CATALOG_NAME, dbname, name, new_parts,
+    return alterPartitions(msdb, wh, MetaStoreUtils.getDefaultCatalog(conf), dbname, name, new_parts,
         environmentContext, null, -1, null);
   }
 
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 dd646db..f5a78a2 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.commons.lang3.StringUtils.join;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
 
@@ -3991,7 +3990,7 @@ public class ObjectStore implements RawStore, Configurable {
           SqlFilterForPushdown filter = new SqlFilterForPushdown();
           if (directSql.generateSqlFilterForPushdown(catName, dbName, tblName, partitionKeys,
               exprTree, defaultPartitionName, filter)) {
-            String catalogName = (catName != null) ? catName : DEFAULT_CATALOG_NAME;
+            String catalogName = (catName != null) ? catName : getDefaultCatalog(conf);
             return directSql.getPartitionsViaSqlFilter(catalogName, dbName, tblName, filter, null, isAcidTable);
           }
         }
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
index bc38050..51eddd7 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
@@ -53,8 +53,6 @@ import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
 import org.apache.hadoop.hive.metastore.model.MTable;
 import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
 
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
-
 /**
  * This class contains conversion logic that creates Thrift stat objects from
  * JDO stat objects and plain arrays from DirectSQL.
@@ -72,7 +70,7 @@ public class StatObjectConverter {
      MTableColumnStatistics mColStats = new MTableColumnStatistics();
      mColStats.setTable(table);
      mColStats.setDbName(statsDesc.getDbName());
-     mColStats.setCatName(statsDesc.isSetCatName() ? statsDesc.getCatName() : DEFAULT_CATALOG_NAME);
+     mColStats.setCatName(table.getDatabase().getCatalogName());
      mColStats.setTableName(statsDesc.getTableName());
      mColStats.setLastAnalyzed(statsDesc.getLastAnalyzed());
      mColStats.setColName(statsObj.getColName());
@@ -450,8 +448,13 @@ public class StatObjectConverter {
     }
 
     MPartitionColumnStatistics mColStats = new MPartitionColumnStatistics();
-    mColStats.setPartition(partition);
-    mColStats.setCatName(statsDesc.isSetCatName() ? statsDesc.getCatName() : DEFAULT_CATALOG_NAME);
+    if (partition != null) {
+      mColStats.setCatName(partition.getTable().getDatabase().getCatalogName());
+      mColStats.setPartition(partition);
+    } else {
+      // Assume that the statsDesc has already set catalogName when partition is null
+      mColStats.setCatName(statsDesc.getCatName());
+    }
     mColStats.setDbName(statsDesc.getDbName());
     mColStats.setTableName(statsDesc.getTableName());
     mColStats.setPartitionName(statsDesc.getPartName());
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 09a6051..c04230c 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
@@ -76,7 +76,6 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 
 import static org.apache.hadoop.hive.metastore.HMSHandler.getPartValsFromName;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
 import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
 
@@ -1321,7 +1320,7 @@ public class CachedStore implements RawStore, Configurable {
     if (succ && !canUseEvents) {
       String dbName = normalizeIdentifier(part.getDbName());
       String tblName = normalizeIdentifier(part.getTableName());
-      String catName = part.isSetCatName() ? normalizeIdentifier(part.getCatName()) : DEFAULT_CATALOG_NAME;
+      String catName = part.isSetCatName() ? normalizeIdentifier(part.getCatName()) : getDefaultCatalog(conf);
       if (!shouldCacheTable(catName, dbName, tblName)) {
         return succ;
       }
@@ -2191,7 +2190,7 @@ public class CachedStore implements RawStore, Configurable {
   private void updatePartitionColumnStatisticsInCache(ColumnStatistics colStats, Map<String, String> newParams,
                                                   List<String> partVals) throws MetaException, NoSuchObjectException {
     String catName = colStats.getStatsDesc().isSetCatName() ? normalizeIdentifier(
-            colStats.getStatsDesc().getCatName()) : DEFAULT_CATALOG_NAME;
+            colStats.getStatsDesc().getCatName()) : getDefaultCatalog(conf);
     String dbName = normalizeIdentifier(colStats.getStatsDesc().getDbName());
     String tblName = normalizeIdentifier(colStats.getStatsDesc().getTableName());
     if (!shouldCacheTable(catName, dbName, tblName)) {
@@ -2826,7 +2825,7 @@ public class CachedStore implements RawStore, Configurable {
     }
     String dbName = normalizeIdentifier(tbl.getDbName());
     String tblName = normalizeIdentifier(tbl.getTableName());
-    String catName = tbl.isSetCatName() ? normalizeIdentifier(tbl.getCatName()) : DEFAULT_CATALOG_NAME;
+    String catName = tbl.isSetCatName() ? normalizeIdentifier(tbl.getCatName()) : getDefaultCatalog(conf);
     if (!shouldCacheTable(catName, dbName, tblName)) {
       return constraints;
     }
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
index bf65d1b..cfcab47 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/VerifyingObjectStore.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.commons.lang3.StringUtils.repeat;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
 import java.lang.reflect.AccessibleObject;
 import java.lang.reflect.Array;