You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2021/09/03 05:35:04 UTC

[incubator-doris] branch master updated: [Catalog] Enforce null check at Catalog.getDb and Database.getTable (#6416)

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

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new df54b34  [Catalog] Enforce null check at Catalog.getDb and Database.getTable (#6416)
df54b34 is described below

commit df54b34f9843adf80d5fa753383797690e9458dd
Author: ccoffline <45...@users.noreply.github.com>
AuthorDate: Fri Sep 3 13:34:49 2021 +0800

    [Catalog] Enforce null check at Catalog.getDb and Database.getTable (#6416)
    
    fix #5378 #5391 #5688 #5973 #6155 and all replay NPE. All replay method can now throw MetaNotFoundException and caught to log a warning for potential inconsistent metadata cases.
    
    try to establish a clear notice for future developer to check null.
---
 .../main/java/org/apache/doris/alter/Alter.java    |  80 +--
 .../java/org/apache/doris/alter/AlterHandler.java  |  61 +-
 .../java/org/apache/doris/alter/AlterJobV2.java    |   6 +-
 .../doris/alter/MaterializedViewHandler.java       |  78 +--
 .../java/org/apache/doris/alter/RollupJob.java     |  58 +-
 .../java/org/apache/doris/alter/RollupJobV2.java   | 133 ++--
 .../apache/doris/alter/SchemaChangeHandler.java    |  24 +-
 .../org/apache/doris/alter/SchemaChangeJob.java    |  83 +--
 .../org/apache/doris/alter/SchemaChangeJobV2.java  | 116 ++--
 .../org/apache/doris/analysis/AlterViewStmt.java   |   4 +-
 .../java/org/apache/doris/analysis/Analyzer.java   |  20 +-
 .../doris/analysis/CreateDataSyncJobStmt.java      |  20 +-
 .../doris/analysis/CreateRoutineLoadStmt.java      |  10 +-
 .../org/apache/doris/analysis/DataDescription.java |  22 +-
 .../org/apache/doris/analysis/DescribeStmt.java    |  11 +-
 .../org/apache/doris/analysis/EncryptKeyRef.java   |   5 +-
 .../java/org/apache/doris/analysis/ExportStmt.java |  12 +-
 .../java/org/apache/doris/analysis/FromClause.java |  11 +-
 .../apache/doris/analysis/FunctionCallExpr.java    |   2 +-
 .../java/org/apache/doris/analysis/InsertStmt.java |  17 +-
 .../java/org/apache/doris/analysis/LoadStmt.java   |  14 +-
 .../java/org/apache/doris/analysis/SelectStmt.java |  10 +-
 .../org/apache/doris/analysis/ShowAlterStmt.java   |   8 +-
 .../org/apache/doris/analysis/ShowDataStmt.java    |   7 +-
 .../apache/doris/analysis/ShowPartitionsStmt.java  |   7 +-
 .../org/apache/doris/analysis/ShowViewStmt.java    |  17 +-
 .../java/org/apache/doris/analysis/UpdateStmt.java |  12 +-
 .../org/apache/doris/backup/BackupHandler.java     |  15 +-
 .../java/org/apache/doris/backup/BackupJob.java    |  11 +-
 .../java/org/apache/doris/backup/RestoreJob.java   |  57 +-
 .../java/org/apache/doris/catalog/Catalog.java     | 669 +++++++++------------
 .../apache/doris/catalog/CatalogRecycleBin.java    |   4 +-
 .../apache/doris/catalog/ColocateTableIndex.java   |  18 +-
 .../java/org/apache/doris/catalog/Database.java    | 117 ++--
 .../org/apache/doris/catalog/EncryptKeyHelper.java |  27 +-
 .../org/apache/doris/catalog/InfoSchemaDb.java     |   4 +-
 .../org/apache/doris/catalog/MetadataViewer.java   |  51 +-
 .../org/apache/doris/catalog/TabletStatMgr.java    |   2 +-
 .../clone/ColocateTableCheckerAndBalancer.java     |   6 +-
 .../doris/clone/DynamicPartitionScheduler.java     |   6 +-
 .../java/org/apache/doris/clone/TabletChecker.java |  28 +-
 .../org/apache/doris/clone/TabletSchedCtx.java     |  32 +-
 .../org/apache/doris/clone/TabletScheduler.java    |  13 +-
 .../apache/doris/common/MetaNotFoundException.java |   4 +
 .../org/apache/doris/common/proc/DbsProcDir.java   |   7 +-
 .../apache/doris/common/proc/JobsDbProcDir.java    |   9 +-
 .../apache/doris/common/proc/StatisticProcDir.java |   2 +-
 .../apache/doris/common/proc/TablesProcDir.java    |   9 +-
 .../org/apache/doris/common/util/SmallFileMgr.java |  16 +-
 .../doris/consistency/CheckConsistencyJob.java     |   8 +-
 .../doris/consistency/ConsistencyChecker.java      |   9 +-
 .../doris/external/elasticsearch/EsRepository.java |   5 +-
 .../apache/doris/http/rest/CancelStreamLoad.java   |   5 +-
 .../apache/doris/http/rest/GetDdlStmtAction.java   |  11 +-
 .../apache/doris/http/rest/GetStreamLoadState.java |   5 +-
 .../apache/doris/http/rest/MigrationAction.java    |  16 +-
 .../org/apache/doris/http/rest/RowCountAction.java |  23 +-
 .../apache/doris/http/rest/ShowMetaInfoAction.java |  13 +-
 .../doris/http/rest/StorageTypeCheckAction.java    |   5 +-
 .../doris/http/rest/TableQueryPlanAction.java      |  26 +-
 .../doris/http/rest/TableRowCountAction.java       |  21 +-
 .../apache/doris/http/rest/TableSchemaAction.java  |  22 +-
 .../apache/doris/httpv2/rest/CancelLoadAction.java |   9 +-
 .../apache/doris/httpv2/rest/GetDdlStmtAction.java |  15 +-
 .../doris/httpv2/rest/GetStreamLoadState.java      |   9 +-
 .../apache/doris/httpv2/rest/MetaInfoAction.java   |  26 +-
 .../apache/doris/httpv2/rest/RowCountAction.java   |  10 +-
 .../org/apache/doris/httpv2/rest/ShowAction.java   |  10 +-
 .../doris/httpv2/rest/StorageTypeCheckAction.java  |   9 +-
 .../doris/httpv2/rest/TableQueryPlanAction.java    |  10 +-
 .../doris/httpv2/rest/TableRowCountAction.java     |   9 +-
 .../doris/httpv2/rest/TableSchemaAction.java       |   9 +-
 .../doris/httpv2/restv2/MetaInfoActionV2.java      |  44 +-
 .../doris/httpv2/restv2/StatisticAction.java       |  11 +-
 .../org/apache/doris/load/BrokerFileGroup.java     |  27 +-
 .../java/org/apache/doris/load/DeleteHandler.java  |  29 +-
 .../main/java/org/apache/doris/load/DeleteJob.java |   7 +-
 .../main/java/org/apache/doris/load/ExportJob.java |  11 +-
 .../main/java/org/apache/doris/load/ExportMgr.java |   2 +-
 .../src/main/java/org/apache/doris/load/Load.java  | 110 ++--
 .../java/org/apache/doris/load/LoadChecker.java    |   8 +-
 .../org/apache/doris/load/StreamLoadRecordMgr.java |   2 +-
 .../apache/doris/load/loadv2/BrokerLoadJob.java    |   2 +-
 .../org/apache/doris/load/loadv2/BulkLoadJob.java  |  55 +-
 .../apache/doris/load/loadv2/InsertLoadJob.java    |  29 +-
 .../java/org/apache/doris/load/loadv2/LoadJob.java |  12 +-
 .../org/apache/doris/load/loadv2/LoadManager.java  |  39 +-
 .../doris/load/loadv2/LoadingTaskPlanner.java      |   2 +-
 .../org/apache/doris/load/loadv2/MiniLoadJob.java  |   5 +-
 .../doris/load/loadv2/SparkLoadPendingTask.java    |  24 +-
 .../load/routineload/KafkaRoutineLoadJob.java      |  16 +-
 .../doris/load/routineload/KafkaTaskInfo.java      |  11 +-
 .../doris/load/routineload/RoutineLoadJob.java     |  72 +--
 .../doris/load/routineload/RoutineLoadManager.java |  13 +-
 .../java/org/apache/doris/load/sync/SyncJob.java   |   7 +-
 .../org/apache/doris/load/sync/SyncJobManager.java |  24 +-
 .../apache/doris/load/sync/canal/CanalSyncJob.java |  17 +-
 .../doris/load/update/UpdateStmtExecutor.java      |  13 +-
 .../java/org/apache/doris/master/MasterImpl.java   |   9 +-
 .../master/PartitionInMemoryInfoCollector.java     |   2 +-
 .../org/apache/doris/master/ReportHandler.java     |  20 +-
 .../java/org/apache/doris/persist/EditLog.java     |  21 +-
 .../java/org/apache/doris/qe/ConnectContext.java   |   7 +-
 .../java/org/apache/doris/qe/ConnectProcessor.java |   4 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     | 282 ++++-----
 .../java/org/apache/doris/qe/StmtExecutor.java     |  10 +-
 .../apache/doris/service/FrontendServiceImpl.java  |  39 +-
 .../apache/doris/statistics/StatisticsManager.java |  21 +-
 .../org/apache/doris/system/SystemInfoService.java |   4 +-
 .../org/apache/doris/task/ExportPendingTask.java   |   2 +-
 .../apache/doris/task/HadoopLoadPendingTask.java   |  10 +-
 .../java/org/apache/doris/task/LoadEtlTask.java    |  12 +-
 .../org/apache/doris/task/LoadPendingTask.java     |   2 +-
 .../doris/transaction/DatabaseTransactionMgr.java  |  40 +-
 .../transaction/DbUsedDataQuotaInfoCollector.java  |   2 +-
 .../doris/transaction/GlobalTransactionMgr.java    |  15 +-
 .../doris/transaction/PublishVersionDaemon.java    |   4 +-
 .../org/apache/doris/alter/AlterJobV2Test.java     |  13 +-
 .../java/org/apache/doris/alter/AlterTest.java     |  82 +--
 .../org/apache/doris/alter/BatchRollupJobTest.java |   8 +-
 .../org/apache/doris/alter/RollupJobV2Test.java    |  16 +-
 .../apache/doris/alter/SchemaChangeJobV2Test.java  |  16 +-
 .../org/apache/doris/analysis/AccessTestUtil.java  |  28 +-
 .../doris/analysis/AdminShowReplicaTest.java       |   4 +-
 .../apache/doris/analysis/AlterViewStmtTest.java   |  55 +-
 .../doris/analysis/CreateDataSyncJobStmtTest.java  |   4 +-
 .../doris/analysis/CreateRoutineLoadStmtTest.java  |   4 +-
 .../apache/doris/analysis/DataDescriptionTest.java |   4 +-
 .../apache/doris/analysis/GroupByClauseTest.java   |   4 +-
 .../apache/doris/analysis/ShowDataStmtTest.java    |   4 +-
 .../analysis/TableNameComparedLowercaseTest.java   |   2 +-
 .../analysis/TableNameStoredLowercaseTest.java     |   2 +-
 .../apache/doris/analysis/VirtualSlotRefTest.java  |   9 +-
 .../org/apache/doris/backup/BackupHandlerTest.java |  80 +--
 .../org/apache/doris/backup/BackupJobTest.java     |   6 +-
 .../org/apache/doris/backup/CatalogMocker.java     |  12 +-
 .../org/apache/doris/backup/RestoreJobTest.java    |  10 +-
 .../org/apache/doris/catalog/AdminStmtTest.java    |   4 +-
 .../apache/doris/catalog/CatalogOperationTest.java |  18 +-
 .../org/apache/doris/catalog/CatalogTestUtil.java  |  86 +--
 .../apache/doris/catalog/ColocateTableTest.java    |  10 +-
 .../apache/doris/catalog/CreateEncryptKeyTest.java |   2 +-
 .../apache/doris/catalog/CreateFunctionTest.java   |   2 +-
 .../apache/doris/catalog/CreateTableLikeTest.java  |  16 +-
 .../org/apache/doris/catalog/CreateTableTest.java  |   8 +-
 .../org/apache/doris/catalog/CreateViewTest.java   |  12 +-
 .../org/apache/doris/catalog/DatabaseTest.java     |  24 +-
 .../java/org/apache/doris/catalog/DropDbTest.java  |  16 +-
 .../apache/doris/catalog/DropPartitionTest.java    |  12 +-
 .../org/apache/doris/catalog/DropTableTest.java    |  10 +-
 .../doris/catalog/DynamicPartitionTableTest.java   |  46 +-
 .../org/apache/doris/catalog/InfoSchemaDbTest.java |   2 +-
 .../apache/doris/catalog/MetadataViewerTest.java   |   8 +-
 .../java/org/apache/doris/catalog/RecoverTest.java |  30 +-
 .../apache/doris/catalog/TempPartitionTest.java    |  16 +-
 .../java/org/apache/doris/clone/RebalanceTest.java |   9 +-
 .../doris/cluster/SystemInfoServiceTest.java       |   4 +-
 .../apache/doris/common/proc/DbsProcDirTest.java   |  24 +-
 .../apache/doris/common/util/SmallFileMgrTest.java |  11 +-
 .../elasticsearch/EsShardPartitionsTest.java       |   4 +-
 .../org/apache/doris/http/DorisHttpTestCase.java   |   8 +-
 .../doris/http/TableQueryPlanActionTest.java       |   3 +-
 .../org/apache/doris/load/DeleteHandlerTest.java   |  13 +-
 .../org/apache/doris/load/LoadCheckerTest.java     |   8 +-
 .../doris/load/loadv2/BrokerLoadJobTest.java       |  21 +-
 .../doris/load/loadv2/InsertLoadJobTest.java       |   7 +-
 .../org/apache/doris/load/loadv2/LoadJobTest.java  |  20 -
 .../apache/doris/load/loadv2/LoadManagerTest.java  |  10 +-
 .../apache/doris/load/loadv2/SparkLoadJobTest.java |  63 +-
 .../load/loadv2/SparkLoadPendingTaskTest.java      |  57 --
 .../load/routineload/KafkaRoutineLoadJobTest.java  |  26 +-
 .../doris/load/routineload/RoutineLoadJobTest.java |  10 +-
 .../load/routineload/RoutineLoadManagerTest.java   |  10 +-
 .../load/routineload/RoutineLoadSchedulerTest.java |   6 +-
 .../apache/doris/load/sync/SyncJobManagerTest.java |   2 +-
 .../doris/load/sync/canal/CanalSyncJobTest.java    |   6 +-
 .../doris/load/update/UpdateStmtExecutorTest.java  |   2 -
 .../org/apache/doris/mysql/MysqlProtoTest.java     |   2 +-
 .../apache/doris/persist/LoadJobV2PersistTest.java |   4 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |  16 +-
 .../org/apache/doris/planner/RepeatNodeTest.java   |   6 +-
 .../doris/planner/StreamLoadScanNodeTest.java      |   4 +-
 .../org/apache/doris/qe/PartitionCacheTest.java    |   6 +-
 .../java/org/apache/doris/qe/ShowExecutorTest.java |  10 +-
 .../org/apache/doris/task/LoadEtlTaskTest.java     |   6 +-
 .../org/apache/doris/task/LoadPendingTaskTest.java |   6 +-
 .../transaction/DatabaseTransactionMgrTest.java    |   2 +-
 .../transaction/GlobalTransactionMgrTest.java      |  24 +-
 .../org/apache/doris/utframe/AnotherDemoTest.java  |  22 +-
 .../java/org/apache/doris/utframe/DemoTest.java    |   8 +-
 190 files changed, 1731 insertions(+), 2732 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
index 9338552..fe3537a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
@@ -50,8 +50,6 @@ import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.View;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.DynamicPartitionUtil;
@@ -99,32 +97,25 @@ public class Alter {
         String tableName = stmt.getBaseIndexName();
         // check db
         String dbName = stmt.getDBName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         // check cluster capacity
         Catalog.getCurrentSystemInfo().checkClusterCapacity(stmt.getClusterName());
         // check db quota
         db.checkQuota();
 
-        OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP);
-        ((MaterializedViewHandler)materializedViewHandler).processCreateMaterializedView(stmt, db,
-                    olapTable);
+        OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
+        ((MaterializedViewHandler) materializedViewHandler).processCreateMaterializedView(stmt, db, olapTable);
     }
 
     public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException {
         // check db
         String dbName = stmt.getTableName().getDb();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         String tableName = stmt.getTableName().getTbl();
-        OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP);
+        OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
         // drop materialized view
-        ((MaterializedViewHandler)materializedViewHandler).processDropMaterializedView(stmt, db, olapTable);
+        ((MaterializedViewHandler) materializedViewHandler).processDropMaterializedView(stmt, db, olapTable);
     }
 
     private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable, List<AlterClause> alterClauses,
@@ -260,17 +251,11 @@ public class Alter {
         }
     }
 
-    public void replayModifyComment(ModifyCommentOperationLog operation) {
+    public void replayModifyComment(ModifyCommentOperationLog operation) throws MetaNotFoundException {
         long dbId = operation.getDbId();
         long tblId = operation.getTblId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            return;
-        }
-        Table tbl = db.getTable(tblId);
-        if (tbl == null) {
-            return;
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        Table tbl = db.getTableOrMetaException(tblId);
         tbl.writeLock();
         try {
             ModifyCommentOperationLog.Type type = operation.getType();
@@ -311,14 +296,8 @@ public class Alter {
         String tableName = dbTableName.getTbl();
         final String clusterName = stmt.getClusterName();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
+        Table table = db.getTableOrDdlException(tableName);
         List<AlterClause> alterClauses = Lists.newArrayList();
         // some operations will take long time to process, need to be done outside the table lock
         boolean needProcessOutsideTableLock = false;
@@ -342,7 +321,7 @@ public class Alter {
             AlterClause alterClause = alterClauses.get(0);
             if (alterClause instanceof AddPartitionClause) {
                 if (!((AddPartitionClause) alterClause).isTempPartition()) {
-                    DynamicPartitionUtil.checkAlterAllowed((OlapTable) db.getTable(tableName));
+                    DynamicPartitionUtil.checkAlterAllowed((OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP));
                 }
                 Catalog.getCurrentCatalog().addPartition(db, tableName, (AddPartitionClause) alterClause);
             } else if (alterClause instanceof ModifyPartitionClause) {
@@ -376,7 +355,7 @@ public class Alter {
         ReplaceTableClause clause = (ReplaceTableClause) alterClauses.get(0);
         String newTblName = clause.getTblName();
         boolean swapTable = clause.isSwapTable();
-        Table newTbl = db.getTableOrThrowException(newTblName, TableType.OLAP);
+        Table newTbl = db.getTableOrMetaException(newTblName, TableType.OLAP);
         OlapTable olapNewTbl = (OlapTable) newTbl;
         db.writeLock();
         origTable.writeLock();
@@ -399,13 +378,14 @@ public class Alter {
 
     }
 
-    public void replayReplaceTable(ReplaceTableOperationLog log) {
+    public void replayReplaceTable(ReplaceTableOperationLog log) throws MetaNotFoundException {
         long dbId = log.getDbId();
         long origTblId = log.getOrigTblId();
         long newTblId = log.getNewTblId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        OlapTable origTable = (OlapTable) db.getTable(origTblId);
-        OlapTable newTbl = (OlapTable) db.getTable(newTblId);
+
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable origTable = db.getTableOrMetaException(origTblId, TableType.OLAP);
+        OlapTable newTbl = db.getTableOrMetaException(newTblId, TableType.OLAP);
 
         try {
             replaceTableInternal(db, origTable, newTbl, log.isSwapTable(), true);
@@ -457,13 +437,10 @@ public class Alter {
         TableName dbTableName = stmt.getTbl();
         String dbName = dbTableName.getDb();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         String tableName = dbTableName.getTbl();
-        View view = (View) db.getTableOrThrowException(tableName, TableType.VIEW);
+        View view = db.getTableOrMetaException(tableName, TableType.VIEW);
         modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns());
     }
 
@@ -491,14 +468,15 @@ public class Alter {
         }
     }
 
-    public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws DdlException {
+    public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws MetaNotFoundException, DdlException {
         long dbId = alterViewInfo.getDbId();
         long tableId = alterViewInfo.getTableId();
         String inlineViewDef = alterViewInfo.getInlineViewDef();
         List<Column> newFullSchema = alterViewInfo.getNewFullSchema();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        View view = (View) db.getTable(tableId);
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        View view = db.getTableOrMetaException(tableId, TableType.VIEW);
+
         db.writeLock();
         view.writeLock();
         try {
@@ -631,13 +609,9 @@ public class Alter {
         Catalog.getCurrentCatalog().getEditLog().logBatchModifyPartition(info);
     }
 
-    public void replayModifyPartition(ModifyPartitionInfo info) {
-        Database db = Catalog.getCurrentCatalog().getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
-        if (olapTable == null) {
-            LOG.warn("table {} does not eixst when replaying modify partition. db: {}", info.getTableId(), info.getDbId());
-            return;
-        }
+    public void replayModifyPartition(ModifyPartitionInfo info) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             PartitionInfo partitionInfo = olapTable.getPartitionInfo();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java
index 61fdcca..2f825b2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterHandler.java
@@ -306,41 +306,49 @@ public abstract class AlterHandler extends MasterDaemon {
         }
     }
 
-    public void replayInitJob(AlterJob alterJob, Catalog catalog) {
-        Database db = catalog.getDb(alterJob.getDbId());
-        alterJob.replayInitJob(db);
-        // add rollup job
-        addAlterJob(alterJob);
+    public void replayInitJob(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException {
+        try {
+            Database db = catalog.getDbOrMetaException(alterJob.getDbId());
+            alterJob.replayInitJob(db);
+        } finally {
+            // add rollup job
+            addAlterJob(alterJob);
+        }
     }
     
-    public void replayFinishing(AlterJob alterJob, Catalog catalog) {
-        Database db = catalog.getDb(alterJob.getDbId());
-        alterJob.replayFinishing(db);
-        alterJob.setState(JobState.FINISHING);
-        // !!! the alter job should add to the cache again, because the alter job is deserialized from journal
-        // it is a different object compared to the cache
-        addAlterJob(alterJob);
+    public void replayFinishing(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException {
+        try {
+            Database db = catalog.getDbOrMetaException(alterJob.getDbId());
+            alterJob.replayFinishing(db);
+        } finally {
+            alterJob.setState(JobState.FINISHING);
+            // !!! the alter job should add to the cache again, because the alter job is deserialized from journal
+            // it is a different object compared to the cache
+            addAlterJob(alterJob);
+        }
     }
 
-    public void replayFinish(AlterJob alterJob, Catalog catalog) {
-        Database db = catalog.getDb(alterJob.getDbId());
-        alterJob.replayFinish(db);
-        alterJob.setState(JobState.FINISHED);
-
-        jobDone(alterJob);
+    public void replayFinish(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException {
+        try {
+            Database db = catalog.getDbOrMetaException(alterJob.getDbId());
+            alterJob.replayFinish(db);
+        } finally {
+            alterJob.setState(JobState.FINISHED);
+            jobDone(alterJob);
+        }
     }
 
-    public void replayCancel(AlterJob alterJob, Catalog catalog) {
+    public void replayCancel(AlterJob alterJob, Catalog catalog) throws MetaNotFoundException {
         removeAlterJob(alterJob.getTableId());
         alterJob.setState(JobState.CANCELLED);
-        Database db = catalog.getDb(alterJob.getDbId());
-        if (db != null) {
+        try {
             // we log rollup job cancelled even if db is dropped.
             // so check db != null here
+            Database db = catalog.getDbOrMetaException(alterJob.getDbId());
             alterJob.replayCancel(db);
+        } finally {
+            addFinishedOrCancelledAlterJob(alterJob);
         }
-
-        addFinishedOrCancelledAlterJob(alterJob);
     }
 
     @Override
@@ -409,12 +417,9 @@ public abstract class AlterHandler extends MasterDaemon {
      * In summary, we only need to update replica's version when replica's version is smaller than X
      */
     public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundException {
-        Database db = Catalog.getCurrentCatalog().getDb(task.getDbId());
-        if (db == null) {
-            throw new MetaNotFoundException("database " + task.getDbId() + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(task.getDbId());
 
-        OlapTable tbl = (OlapTable) db.getTableOrThrowException(task.getTableId(), Table.TableType.OLAP);
+        OlapTable tbl = db.getTableOrMetaException(task.getTableId(), Table.TableType.OLAP);
         tbl.writeLock();
         try {
             Partition partition = tbl.getPartition(task.getPartitionId());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
index 15d5a39..6a31be1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
@@ -187,9 +187,9 @@ public abstract class AlterJobV2 implements Writable {
     * return false if table is not stable.
     */
     protected boolean checkTableStable(Database db) throws AlterCancelException {
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -206,7 +206,7 @@ public abstract class AlterJobV2 implements Writable {
                 return false;
             } else {
                 // table is stable, set is to ROLLUP and begin altering.
-                LOG.info("table {} is stable, start {} job {}", tableId, type);
+                LOG.info("table {} is stable, start {} job {}", tableId, type, jobId);
                 tbl.setState(type == JobType.ROLLUP ? OlapTableState.ROLLUP : OlapTableState.SCHEMA_CHANGE);
                 errMsg = "";
                 return true;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
index 7fab577..f1d231a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
@@ -45,8 +45,6 @@ import org.apache.doris.catalog.TabletMeta;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.util.ListComparator;
@@ -827,17 +825,14 @@ public class MaterializedViewHandler extends AlterHandler {
         return mvIndexId;
     }
 
-    public void replayDropRollup(DropInfo dropInfo, Catalog catalog) {
-        Database db = catalog.getDb(dropInfo.getDbId());
+    public void replayDropRollup(DropInfo dropInfo, Catalog catalog) throws MetaNotFoundException {
+        long dbId = dropInfo.getDbId();
         long tableId = dropInfo.getTableId();
         long rollupIndexId = dropInfo.getIndexId();
 
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
-        if (olapTable == null) {
-            LOG.warn("table {} does not exist when replaying drop rollup. db: {}", tableId, db.getId());
-            return;
-        }
+        Database db = catalog.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         olapTable.writeLock();
         try {
             for (Partition partition : olapTable.getPartitions()) {
@@ -883,26 +878,20 @@ public class MaterializedViewHandler extends AlterHandler {
     }
 
     private void changeTableStatus(long dbId, long tableId, OlapTableState olapTableState) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            LOG.warn("db {} has been dropped when changing table {} status after rollup job done",
-                    dbId, tableId);
-            return;
-        }
-        OlapTable tbl = (OlapTable) db.getTable(tableId);
-        if (tbl == null) {
-            LOG.warn("table {} has been dropped when changing table status after rollup job done",
-                    tableId);
-            return;
-        }
-        tbl.writeLock();
         try {
-            if (tbl.getState() == olapTableState) {
-                return;
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+            OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            olapTable.writeLock();
+            try {
+                if (olapTable.getState() == olapTableState) {
+                    return;
+                }
+                olapTable.setState(olapTableState);
+            } finally {
+                olapTable.writeUnlock();
             }
-            tbl.setState(olapTableState);
-        } finally {
-            tbl.writeUnlock();
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] changing table status failed after rollup job done", e);
         }
     }
 
@@ -913,10 +902,8 @@ public class MaterializedViewHandler extends AlterHandler {
         if (!alterJob.isDone()) {
             addAlterJobV2ToTableNotFinalStateJobMap(alterJob);
             changeTableStatus(alterJob.getDbId(), alterJob.getTableId(), OlapTableState.ROLLUP);
-        } else {
-            if (removeAlterJobV2FromTableNotFinalStateJobMap(alterJob)) {
-                changeTableStatus(alterJob.getDbId(), alterJob.getTableId(), OlapTableState.NORMAL);
-            }
+        } else if (removeAlterJobV2FromTableNotFinalStateJobMap(alterJob)) {
+            changeTableStatus(alterJob.getDbId(), alterJob.getTableId(), OlapTableState.NORMAL);
         }
     }
 
@@ -1079,13 +1066,13 @@ public class MaterializedViewHandler extends AlterHandler {
 
         // handle cancelled rollup jobs
         for (AlterJob rollupJob : cancelledJobs) {
-            Database db = Catalog.getCurrentCatalog().getDb(rollupJob.getDbId());
+            Database db = Catalog.getCurrentCatalog().getDbNullable(rollupJob.getDbId());
             if (db == null) {
                 cancelInternal(rollupJob, null, null);
                 continue;
             }
 
-            OlapTable olapTable = (OlapTable) db.getTable(rollupJob.getTableId());
+            OlapTable olapTable = (OlapTable) db.getTableNullable(rollupJob.getTableId());
             if (olapTable != null) {
                 olapTable.writeLock();
             }
@@ -1165,15 +1152,15 @@ public class MaterializedViewHandler extends AlterHandler {
 
 
         for (AlterJob selectedJob : jobs) {
-            OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId());
-            if (olapTable == null) {
-                continue;
-            }
-            olapTable.readLock();
             try {
-                selectedJob.getJobInfo(rollupJobInfos, olapTable);
-            } finally {
-                olapTable.readUnlock();
+                OlapTable olapTable = db.getTableOrMetaException(selectedJob.getTableId(), Table.TableType.OLAP);
+                olapTable.readLock();
+                try {
+                    selectedJob.getJobInfo(rollupJobInfos, olapTable);
+                } finally {
+                    olapTable.readUnlock();
+                }
+            } catch (MetaNotFoundException ignored) {
             }
 
         }
@@ -1203,16 +1190,13 @@ public class MaterializedViewHandler extends AlterHandler {
         Preconditions.checkState(!Strings.isNullOrEmpty(dbName));
         Preconditions.checkState(!Strings.isNullOrEmpty(tableName));
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         AlterJob rollupJob = null;
         List<AlterJobV2> rollupJobV2List = new ArrayList<>();
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new DdlException(e.getMessage());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
index 8a7830a..7195d58 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
@@ -341,7 +341,7 @@ public class RollupJob extends AlterJob {
         if (!clearFailed && batchClearAlterTask != null) {
             return 1;
         }
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = "db[" + dbId + "] does not exist";
             LOG.warn(cancelMsg);
@@ -351,9 +351,9 @@ public class RollupJob extends AlterJob {
         batchClearAlterTask = new AgentBatchTask();
 
         synchronized (this) {
-            OlapTable olapTable = null;
+            OlapTable olapTable;
             try {
-                olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+                olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
             } catch (MetaNotFoundException e) {
                 LOG.warn(e.getMessage());
                 return -1;
@@ -410,7 +410,7 @@ public class RollupJob extends AlterJob {
         // here we just rejoin tasks to AgentTaskQueue.
         // task report process will later resend these tasks
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = "db[" + dbId + "] does not exist";
             LOG.warn(cancelMsg);
@@ -419,9 +419,9 @@ public class RollupJob extends AlterJob {
 
 
         synchronized (this) {
-            OlapTable olapTable = null;
+            OlapTable olapTable;
             try {
-                olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+                olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
             } catch (MetaNotFoundException e) {
                 LOG.warn(e.getMessage());
                 return false;
@@ -628,16 +628,16 @@ public class RollupJob extends AlterJob {
             return 0;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = "Db[" + dbId + "] does not exist";
             LOG.warn(cancelMsg);
             return -1;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return -1;
@@ -788,7 +788,13 @@ public class RollupJob extends AlterJob {
 
     @Override
     public void replayInitJob(Database db) {
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay init rollup job failed", e);
+            return;
+        }
         olapTable.writeLock();
         try {
             // set state
@@ -824,8 +830,14 @@ public class RollupJob extends AlterJob {
 
     @Override
     public void replayFinishing(Database db) {
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay finishing rollup job failed", e);
+            return;
+        }
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
         olapTable.writeLock();
         try {
             for (Map.Entry<Long, MaterializedIndex> entry : this.partitionIdToRollupIndex.entrySet()) {
@@ -908,7 +920,13 @@ public class RollupJob extends AlterJob {
             replayFinishing(db);
         }
 
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay finish rollup job failed", e);
+            return;
+        }
         olapTable.writeLock();
         try {
             olapTable.setState(OlapTableState.NORMAL);
@@ -919,7 +937,13 @@ public class RollupJob extends AlterJob {
 
     @Override
     public void replayCancel(Database db) {
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay cancel rollup job failed", e);
+            return;
+        }
         olapTable.writeLock();
         try{
             if (!Catalog.isCheckpointThread()) {
@@ -943,16 +967,16 @@ public class RollupJob extends AlterJob {
 
     @Override
     public void finishJob() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = String.format("database %d does not exist", dbId);
             LOG.warn(cancelMsg);
             return;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return;
@@ -987,7 +1011,7 @@ public class RollupJob extends AlterJob {
         // base index and rollup index name
         jobInfo.add(baseIndexName);
         jobInfo.add(rollupIndexName);
-        
+
         // rollup id
         jobInfo.add(rollupIndexId);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
index dbd0204..ad410b7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
@@ -104,7 +104,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
     private long rollupIndexId;
     @SerializedName(value = "baseIndexName")
     private String baseIndexName;
-    @SerializedName(value =  "rollupIndexName")
+    @SerializedName(value = "rollupIndexName")
     private String rollupIndexName;
 
     @SerializedName(value = "rollupSchema")
@@ -181,11 +181,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         Preconditions.checkState(jobState == JobState.PENDING, jobState);
 
         LOG.info("begin to send create rollup replica tasks. job: {}", jobId);
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Database " + dbId + " does not exist");
-        }
-
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Database " + s + " does not exist"));
         if (!checkTableStable(db)) {
             return;
         }
@@ -200,9 +196,9 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
             }
         }
         MarkedCountDownLatch<Long, Long> countDownLatch = new MarkedCountDownLatch<Long, Long>(totalReplicaNum);
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -311,7 +307,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         }
 
         tbl.setIndexMeta(rollupIndexId, rollupIndexName, rollupSchema, 0 /* init schema version */,
-                rollupSchemaHash, rollupShortKeyColumnCount,TStorageType.COLUMN, rollupKeysType, origStmt);
+                rollupSchemaHash, rollupShortKeyColumnCount, TStorageType.COLUMN, rollupKeysType, origStmt);
         tbl.rebuildFullSchema();
     }
 
@@ -335,14 +331,11 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         }
 
         LOG.info("previous transactions are all finished, begin to send rollup tasks. job: {}", jobId);
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Databasee " + dbId + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Databasee " + s + " does not exist"));
 
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -406,18 +399,15 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
     @Override
     protected void runRunningJob() throws AlterCancelException {
         Preconditions.checkState(jobState == JobState.RUNNING, jobState);
-        
+
         // must check if db or table still exist first.
         // or if table is dropped, the tasks will never be finished,
         // and the job will be in RUNNING state forever.
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Databasee " + dbId + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Databasee " + s + " does not exist"));
 
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -521,9 +511,9 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         AgentTaskQueue.removeBatchTask(rollupBatchTask, TTaskType.ALTER);
         // remove all rollup indexes, and set state to NORMAL
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db != null) {
-            OlapTable tbl = (OlapTable) db.getTable(tableId);
+            OlapTable tbl = (OlapTable) db.getTableNullable(tableId);
             if (tbl != null) {
                 tbl.writeLock();
                 try {
@@ -553,24 +543,15 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
      * Should replay all changes before this job's state transfer to PENDING.
      * These changes should be same as changes in RollupHander.processAddRollup()
      */
-    private void replayCreateJob(RollupJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            // database may be dropped before replaying this log. just return
-            return;
-        }
+    private void replayCreateJob(RollupJobV2 replayedJob) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
 
-        OlapTable tbl = (OlapTable) db.getTable(tableId);
-        if (tbl == null) {
-            // table may be dropped before replaying this log. just return
-            return;
-        }
-
-        tbl.writeLock();
+        olapTable.writeLock();
         try {
-            addTabletToInvertedIndex(tbl);
+            addTabletToInvertedIndex(olapTable);
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
 
         // to make sure that this job will run runPendingJob() again to create the rollup replicas
@@ -602,24 +583,14 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
      * Replay job in WAITING_TXN state.
      * Should replay all changes in runPendingJob()
      */
-    private void replayPendingJob(RollupJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            // database may be dropped before replaying this log. just return
-            return;
-        }
-
-        OlapTable tbl = (OlapTable) db.getTable(tableId);
-        if (tbl == null) {
-            // table may be dropped before replaying this log. just return
-            return;
-        }
-
-        tbl.writeLock();
+    private void replayPendingJob(RollupJobV2 replayedJob) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        olapTable.writeLock();
         try {
-            addRollupIndexToCatalog(tbl);
+            addRollupIndexToCatalog(olapTable);
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
 
         // should still be in WAITING_TXN state, so that the alter tasks will be resend again
@@ -634,9 +605,9 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
      * Should replay all changes in runRuningJob()
      */
     private void replayRunningJob(RollupJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db != null) {
-            OlapTable tbl = (OlapTable) db.getTable(tableId);
+            OlapTable tbl = (OlapTable) db.getTableNullable(tableId);
             if (tbl != null) {
                 tbl.writeLock();
                 try {
@@ -667,22 +638,26 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
 
     @Override
     public void replay(AlterJobV2 replayedJob) {
-        RollupJobV2 replayedRollupJob = (RollupJobV2) replayedJob;
-        switch (replayedJob.jobState) {
-            case PENDING:
-                replayCreateJob(replayedRollupJob);
-                break;
-            case WAITING_TXN:
-                replayPendingJob(replayedRollupJob);
-                break;
-            case FINISHED:
-                replayRunningJob(replayedRollupJob);
-                break;
-            case CANCELLED:
-                replayCancelled(replayedRollupJob);
-                break;
-            default:
-                break;
+        try {
+            RollupJobV2 replayedRollupJob = (RollupJobV2) replayedJob;
+            switch (replayedJob.jobState) {
+                case PENDING:
+                    replayCreateJob(replayedRollupJob);
+                    break;
+                case WAITING_TXN:
+                    replayPendingJob(replayedRollupJob);
+                    break;
+                case FINISHED:
+                    replayRunningJob(replayedRollupJob);
+                    break;
+                case CANCELLED:
+                    replayCancelled(replayedRollupJob);
+                    break;
+                default:
+                    break;
+            }
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay rollup job failed {}", replayedJob.getJobId(), e);
         }
     }
 
@@ -714,7 +689,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         if (jobState == JobState.RUNNING) {
             List<AgentTask> tasks = rollupBatchTask.getUnfinishedTasks(limit);
             for (AgentTask agentTask : tasks) {
-                AlterReplicaTask rollupTask = (AlterReplicaTask)agentTask;
+                AlterReplicaTask rollupTask = (AlterReplicaTask) agentTask;
                 List<String> info = Lists.newArrayList();
                 info.add(String.valueOf(rollupTask.getBackendId()));
                 info.add(String.valueOf(rollupTask.getBaseTabletId()));
@@ -812,10 +787,14 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
             return;
         }
         // parse the define stmt to schema
-        SqlParser parser = new SqlParser(new SqlScanner(new StringReader(origStmt.originStmt),
-                                                        SqlModeHelper.MODE_DEFAULT));
+        SqlParser parser = new SqlParser(new SqlScanner(new StringReader(origStmt.originStmt), SqlModeHelper.MODE_DEFAULT));
         ConnectContext connectContext = new ConnectContext();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        } catch (MetaNotFoundException e) {
+            throw new IOException("error happens when parsing create materialized view stmt: " + origStmt, e);
+        }
         String clusterName = db.getClusterName();
         // It's almost impossible that db's cluster name is null, just in case
         // because before user want to create database, he must first enter a cluster which means that cluster is set to current ConnectContext
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index e25a27f..b54847e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -1520,13 +1520,13 @@ public class SchemaChangeHandler extends AlterHandler {
 
         // handle cancelled schema change jobs
         for (AlterJob alterJob : cancelledJobs) {
-            Database db = Catalog.getCurrentCatalog().getDb(alterJob.getDbId());
+            Database db = Catalog.getCurrentCatalog().getDbNullable(alterJob.getDbId());
             if (db == null) {
                 cancelInternal(alterJob, null, null);
                 continue;
             }
 
-            OlapTable olapTable = (OlapTable) db.getTable(alterJob.getTableId());
+            OlapTable olapTable = (OlapTable) db.getTableNullable(alterJob.getTableId());
             if (olapTable != null) {
                 olapTable.writeLock();
             }
@@ -1608,7 +1608,7 @@ public class SchemaChangeHandler extends AlterHandler {
         }
 
         for (AlterJob selectedJob : selectedJobs) {
-            OlapTable olapTable = (OlapTable) db.getTable(selectedJob.getTableId());
+            OlapTable olapTable = (OlapTable) db.getTableNullable(selectedJob.getTableId());
             if (olapTable == null) {
                 continue;
             }
@@ -1788,7 +1788,7 @@ public class SchemaChangeHandler extends AlterHandler {
      */
     public void updateTableInMemoryMeta(Database db, String tableName, Map<String, String> properties) throws UserException {
         List<Partition> partitions = Lists.newArrayList();
-        OlapTable olapTable = (OlapTable)db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         olapTable.readLock();
         try {
             partitions.addAll(olapTable.getPartitions());
@@ -1820,7 +1820,7 @@ public class SchemaChangeHandler extends AlterHandler {
                                              String tableName,
                                              List<String> partitionNames,
                                              Map<String, String> properties) throws DdlException, MetaNotFoundException {
-        OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY));
         if (isInMemory == olapTable.isInMemory()) {
             return;
@@ -1847,7 +1847,7 @@ public class SchemaChangeHandler extends AlterHandler {
                                             boolean isInMemory) throws UserException {
         // be id -> <tablet id,schemaHash>
         Map<Long, Set<Pair<Long, Integer>>> beIdToTabletIdWithHash = Maps.newHashMap();
-        OlapTable olapTable = (OlapTable)db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         olapTable.readLock();
         try {
             Partition partition = olapTable.getPartition(partitionName);
@@ -1927,20 +1927,12 @@ public class SchemaChangeHandler extends AlterHandler {
         Preconditions.checkState(!Strings.isNullOrEmpty(dbName));
         Preconditions.checkState(!Strings.isNullOrEmpty(tableName));
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         AlterJob schemaChangeJob = null;
         AlterJobV2 schemaChangeJobV2 = null;
 
-        OlapTable olapTable = null;
-        try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP);
-        } catch (MetaNotFoundException e) {
-            throw new DdlException(e.getMessage());
-        }
+        OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
         olapTable.writeLock();
         try {
             if (olapTable.getState() != OlapTableState.SCHEMA_CHANGE &&
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
index feb442b..9bde44b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
@@ -218,15 +218,15 @@ public class SchemaChangeJob extends AlterJob {
     }
 
     public void deleteAllTableHistorySchema() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             LOG.warn("db[{}] does not exist", dbId);
             return;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return;
@@ -315,7 +315,7 @@ public class SchemaChangeJob extends AlterJob {
             return 1;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = "db[" + dbId + "] does not exist";
             LOG.warn(cancelMsg);
@@ -323,9 +323,9 @@ public class SchemaChangeJob extends AlterJob {
         }
 
         batchClearAlterTask = new AgentBatchTask();
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return -1;
@@ -378,7 +378,7 @@ public class SchemaChangeJob extends AlterJob {
 
         LOG.info("sending schema change job {}, start txn id: {}", tableId, transactionId);
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             String msg = "db[" + dbId + "] does not exist";
             setMsg(msg);
@@ -386,9 +386,9 @@ public class SchemaChangeJob extends AlterJob {
             return false;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return false;
@@ -584,14 +584,10 @@ public class SchemaChangeJob extends AlterJob {
         long replicaId = schemaChangeTask.getReplicaId();
 
         // update replica's info
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("Cannot find db[" + dbId + "]");
-        }
-
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return;
@@ -662,16 +658,16 @@ public class SchemaChangeJob extends AlterJob {
             return 0;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = String.format("database %d does not exist", dbId);
             LOG.warn(cancelMsg);
             return -1;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return -1;
@@ -880,16 +876,16 @@ public class SchemaChangeJob extends AlterJob {
 
     @Override
     public void finishJob() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             cancelMsg = String.format("database %d does not exist", dbId);
             LOG.warn(cancelMsg);
             return;
         }
 
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             LOG.warn(e.getMessage());
             return;
@@ -919,7 +915,13 @@ public class SchemaChangeJob extends AlterJob {
 
     @Override
     public void replayInitJob(Database db) {
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay init schema change job failed", e);
+            return;
+        }
         olapTable.writeLock();
         try {
             // change the state of table/partition and replica, then add object to related List and Set
@@ -961,7 +963,13 @@ public class SchemaChangeJob extends AlterJob {
 
     @Override
     public void replayFinishing(Database db) {
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay finishing schema change job failed", e);
+            return;
+        }
         olapTable.writeLock();
         try {
             // set the status to normal
@@ -1042,14 +1050,18 @@ public class SchemaChangeJob extends AlterJob {
             replayFinishing(db);
         }
 
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
-        if (olapTable != null) {
-            olapTable.writeLock();
-            try {
-                olapTable.setState(OlapTableState.NORMAL);
-            } finally {
-                olapTable.writeUnlock();
-            }
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay finish schema change job failed", e);
+            return;
+        }
+        olapTable.writeLock();
+        try {
+            olapTable.setState(OlapTableState.NORMAL);
+        } finally {
+            olapTable.writeUnlock();
         }
         LOG.info("replay finish schema change job: {}", tableId);
     }
@@ -1057,8 +1069,11 @@ public class SchemaChangeJob extends AlterJob {
     @Override
     public void replayCancel(Database db) {
         // restore partition's state
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
-        if (olapTable == null) {
+        OlapTable olapTable;
+        try {
+            olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay cancel schema change job failed", e);
             return;
         }
         olapTable.writeLock();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
index 3292d86..b5bd9a7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
@@ -198,10 +198,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
     protected void runPendingJob() throws AlterCancelException {
         Preconditions.checkState(jobState == JobState.PENDING, jobState);
         LOG.info("begin to send create replica tasks. job: {}", jobId);
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Databasee " + dbId + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Database " + s + " does not exist"));
 
         if (!checkTableStable(db)) {
             return;
@@ -218,9 +215,9 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
         }
         MarkedCountDownLatch<Long, Long> countDownLatch = new MarkedCountDownLatch<>(totalReplicaNum);
 
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -370,14 +367,11 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
         }
 
         LOG.info("previous transactions are all finished, begin to send schema change tasks. job: {}", jobId);
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Databasee " + dbId + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Database " + s + " does not exist"));
 
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -446,14 +440,11 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
         // must check if db or table still exist first.
         // or if table is dropped, the tasks will never be finished,
         // and the job will be in RUNNING state forever.
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new AlterCancelException("Database " + dbId + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new AlterCancelException("Database " + s + " does not exist"));
 
-        OlapTable tbl = null;
+        OlapTable tbl;
         try {
-            tbl = (OlapTable) db.getTableOrThrowException(tableId, TableType.OLAP);
+            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -623,9 +614,9 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
         AgentTaskQueue.removeBatchTask(schemaChangeBatchTask, TTaskType.ALTER);
         // remove all shadow indexes, and set state to NORMAL
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db != null) {
-            OlapTable tbl = (OlapTable) db.getTable(tableId);
+            OlapTable tbl = (OlapTable) db.getTableNullable(tableId);
             if (tbl != null) {
                 tbl.writeLock();
                 try {
@@ -665,20 +656,10 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
      * Should replay all changes before this job's state transfer to PENDING.
      * These changes should be same as changes in SchemaChangeHandler.createJob()
      */
-    private void replayCreateJob(SchemaChangeJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            // database may be dropped before replaying this log. just return
-            return;
-        }
-
-        OlapTable tbl = (OlapTable) db.getTable(tableId);
-        if (tbl == null) {
-            // table may be dropped before replaying this log. just return
-            return;
-        }
-
-        tbl.writeLock();
+    private void replayCreateJob(SchemaChangeJobV2 replayedJob) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
         try {
             TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
             for (Cell<Long, Long, MaterializedIndex> cell : partitionIndexMap.cellSet()) {
@@ -686,7 +667,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
                 long shadowIndexId = cell.getColumnKey();
                 MaterializedIndex shadowIndex = cell.getValue();
 
-                TStorageMedium medium = tbl.getPartitionInfo().getDataProperty(partitionId).getStorageMedium();
+                TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium();
                 TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId, shadowIndexId,
                         indexSchemaVersionAndHashMap.get(shadowIndexId).schemaHash, medium);
 
@@ -699,9 +680,9 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
             }
             
             // set table state
-            tbl.setState(OlapTableState.SCHEMA_CHANGE);
+            olapTable.setState(OlapTableState.SCHEMA_CHANGE);
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
         
         this.watershedTxnId = replayedJob.watershedTxnId;
@@ -713,23 +694,14 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
      * Replay job in WAITING_TXN state.
      * Should replay all changes in runPendingJob()
      */
-    private void replayPendingJob(SchemaChangeJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            // database may be dropped before replaying this log. just return
-            return;
-        }
-
-        OlapTable tbl = (OlapTable) db.getTable(tableId);
-        if (tbl == null) {
-            // table may be dropped before replaying this log. just return
-            return;
-        }
-        tbl.writeLock();
+    private void replayPendingJob(SchemaChangeJobV2 replayedJob) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
         try {
-            addShadowIndexToCatalog(tbl);
+            addShadowIndexToCatalog(olapTable);
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
 
         // should still be in WAITING_TXN state, so that the alter tasks will be resend again
@@ -743,9 +715,9 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
      * Should replay all changes in runRunningJob()
      */
     private void replayRunningJob(SchemaChangeJobV2 replayedJob) {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db != null) {
-            OlapTable tbl = (OlapTable) db.getTable(tableId);
+            OlapTable tbl = (OlapTable) db.getTableNullable(tableId);
             if (tbl != null) {
                 tbl.writeLock();
                 try {
@@ -774,22 +746,26 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
 
     @Override
     public void replay(AlterJobV2 replayedJob) {
-        SchemaChangeJobV2 replayedSchemaChangeJob = (SchemaChangeJobV2) replayedJob;
-        switch (replayedJob.jobState) {
-            case PENDING:
-                replayCreateJob(replayedSchemaChangeJob);
-                break;
-            case WAITING_TXN:
-                replayPendingJob(replayedSchemaChangeJob);
-                break;
-            case FINISHED:
-                replayRunningJob(replayedSchemaChangeJob);
-                break;
-            case CANCELLED:
-                replayCancelled(replayedSchemaChangeJob);
-                break;
-            default:
-                break;
+        try {
+            SchemaChangeJobV2 replayedSchemaChangeJob = (SchemaChangeJobV2) replayedJob;
+            switch (replayedJob.jobState) {
+                case PENDING:
+                    replayCreateJob(replayedSchemaChangeJob);
+                    break;
+                case WAITING_TXN:
+                    replayPendingJob(replayedSchemaChangeJob);
+                    break;
+                case FINISHED:
+                    replayRunningJob(replayedSchemaChangeJob);
+                    break;
+                case CANCELLED:
+                    replayCancelled(replayedSchemaChangeJob);
+                    break;
+                default:
+                    break;
+            }
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replay schema change job failed {}", replayedJob.getJobId(), e);
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java
index a388703..d9c7aca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java
@@ -41,7 +41,7 @@ public class AlterViewStmt extends BaseViewStmt {
     }
 
     @Override
-    public void analyze(Analyzer analyzer) throws AnalysisException, UserException {
+    public void analyze(Analyzer analyzer) throws UserException {
         super.analyze(analyzer);
         if (tableName == null) {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_TABLES_USED);
@@ -49,7 +49,7 @@ public class AlterViewStmt extends BaseViewStmt {
         tableName.analyze(analyzer);
 
 
-        Table table = analyzer.getTable(tableName);
+        Table table = analyzer.getTableOrAnalysisException(tableName);
         if (!(table instanceof View)) {
             throw new AnalysisException(String.format("ALTER VIEW not allowed on a table:%s.%s", getDbName(), getTable()));
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
index 2a30367..3c5f7e3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
@@ -539,15 +539,8 @@ public class Analyzer {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
         }
 
-        Database database = globalState.catalog.getDb(dbName);
-        if (database == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-
-        Table table = database.getTable(tableName.getTbl());
-        if (table == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName.getTbl());
-        }
+        Database database = globalState.catalog.getDbOrAnalysisException(dbName);
+        Table table = database.getTableOrAnalysisException(tableName.getTbl());
 
         if (table.getType() == TableType.OLAP && (((OlapTable) table).getState() == OlapTableState.RESTORE
                 || ((OlapTable) table).getState() == OlapTableState.RESTORE_WITH_LOAD)) {
@@ -569,12 +562,9 @@ public class Analyzer {
         }
     }
 
-    public Table getTable(TableName tblName) {
-        Database db = globalState.catalog.getDb(tblName.getDb());
-        if (db == null) {
-            return null;
-        }
-        return db.getTable(tblName.getTbl());
+    public Table getTableOrAnalysisException(TableName tblName) throws AnalysisException {
+        Database db = globalState.catalog.getDbOrAnalysisException(tblName.getDb());
+        return db.getTableOrAnalysisException(tblName.getTbl());
     }
 
     public ExprRewriter getExprRewriter() { return globalState.exprRewriter_; }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDataSyncJobStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDataSyncJobStmt.java
index a2435fe..656f445 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDataSyncJobStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateDataSyncJobStmt.java
@@ -21,7 +21,6 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.OlapTable;
-import org.apache.doris.catalog.Table;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
@@ -98,22 +97,13 @@ public class CreateDataSyncJobStmt extends DdlStmt {
 
         for (ChannelDescription channelDescription : channelDescriptions) {
             channelDescription.analyze(dbName);
-            Database db = Catalog.getCurrentCatalog().getDb(dbName);
-            if (db == null) {
-                throw new AnalysisException("Database: " + dbName + " not found.");
-            }
             String tableName = channelDescription.getTargetTable();
-            Table table = db.getTable(tableName);
-            if (table == null) {
-                throw new AnalysisException("Table: " + tableName + " doesn't exist");
-            }
-            if (!(table instanceof OlapTable)) {
-                throw new AnalysisException("Table: " + tableName + " is not an olap table");
-            }
-            if (((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS) {
-                throw new AnalysisException("Table: " + tableName + " is not a unique table, key type: " + ((OlapTable) table).getKeysType());
+            Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+            OlapTable olapTable = db.getOlapTableOrAnalysisException(tableName);
+            if (olapTable.getKeysType() != KeysType.UNIQUE_KEYS) {
+                throw new AnalysisException("Table: " + tableName + " is not a unique table, key type: " + olapTable.getKeysType());
             }
-            if (!((OlapTable) table).hasDeleteSign()) {
+            if (!olapTable.hasDeleteSign()) {
                 throw new AnalysisException("Table: " + tableName + " don't support batch delete. Please upgrade it to support, see `help alter table`.");
             }
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java
index 292eb84..ec9d58a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java
@@ -324,14 +324,8 @@ public class CreateRoutineLoadStmt extends DdlStmt {
         if (Strings.isNullOrEmpty(tableName)) {
             throw new AnalysisException("Table name should not be null");
         }
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new AnalysisException("database: " + dbName + " not found.");
-        }
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            throw new AnalysisException("table: " + tableName + " not found.");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+        Table table = db.getTableOrAnalysisException(tableName);
         if (mergeType != LoadTask.MergeType.APPEND
                 && (table.getType() != Table.TableType.OLAP
                 || ((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java
index 9f52275..b617672 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DataDescription.java
@@ -23,7 +23,6 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.FunctionSet;
 import org.apache.doris.catalog.OlapTable;
-import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ErrorCode;
@@ -713,29 +712,18 @@ public class DataDescription {
     }
 
     private void analyzeSequenceCol(String fullDbName) throws AnalysisException {
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new AnalysisException("Database[" + fullDbName + "] does not exist");
-        }
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            throw new AnalysisException("Unknown table " + tableName
-                    + " in database " + db.getFullName());
-        }
-        if (!(table instanceof OlapTable)) {
-            throw new AnalysisException("Table " + table.getName() + " is not OlapTable");
-        }
-        OlapTable olapTable = (OlapTable) table;
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(fullDbName);
+        OlapTable olapTable = db.getOlapTableOrAnalysisException(tableName);
         // no sequence column in load and table schema
         if (!hasSequenceCol() && !olapTable.hasSequenceCol()) {
             return;
         }
         // check olapTable schema and sequenceCol
         if (olapTable.hasSequenceCol() && !hasSequenceCol()) {
-            throw new AnalysisException("Table " + table.getName() + " has sequence column, need to specify the sequence column");
+            throw new AnalysisException("Table " + olapTable.getName() + " has sequence column, need to specify the sequence column");
         }
         if (hasSequenceCol() && !olapTable.hasSequenceCol()) {
-            throw new AnalysisException("There is no sequence column in the table " + table.getName());
+            throw new AnalysisException("There is no sequence column in the table " + olapTable.getName());
         }
         // check source sequence column is in parsedColumnExprList or Table base schema
         boolean hasSourceSequenceCol = false;
@@ -756,7 +744,7 @@ public class DataDescription {
             }
         }
         if (!hasSourceSequenceCol) {
-            throw new AnalysisException("There is no sequence column " + sequenceCol + " in the " + table.getName()
+            throw new AnalysisException("There is no sequence column " + sequenceCol + " in the " + olapTable.getName()
                     + " or the COLUMNS and SET clause");
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
index e4ab731..8d7cccf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
@@ -109,15 +109,8 @@ public class DescribeStmt extends ShowStmt {
                                                 dbTableName.getTbl());
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbTableName.getDb());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbTableName.getDb());
-        }
-
-        Table table = db.getTable(dbTableName.getTbl());
-        if (table == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTableName.getTbl());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbTableName.getDb());
+        Table table = db.getTableOrAnalysisException(dbTableName.getTbl());
 
         table.readLock();
         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyRef.java
index 4e2f881..00f1881 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyRef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/EncryptKeyRef.java
@@ -60,10 +60,7 @@ public class EncryptKeyRef extends Expr {
             ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
         } else {
             dbName = ClusterNamespace.getFullName(analyzer.getClusterName(), dbName);
-            Database database = analyzer.getCatalog().getDb(dbName);
-            if (null == database) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-            }
+            Database database = analyzer.getCatalog().getDbOrAnalysisException(dbName);
 
             EncryptKey encryptKey = database.getEncryptKey(encryptKeyName.getKeyName());
             if (encryptKey != null) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
index 0810b44..7b1eec3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
@@ -189,16 +189,8 @@ public class ExportStmt extends StatementBase {
     }
 
     private void checkTable(Catalog catalog) throws AnalysisException {
-        Database db = catalog.getDb(tblName.getDb());
-        if (db == null) {
-            throw new AnalysisException("Db does not exist. name: " + tblName.getDb());
-        }
-
-        Table table = db.getTable(tblName.getTbl());
-        if (table == null) {
-            throw new AnalysisException("Table[" + tblName.getTbl() + "] does not exist");
-        }
-
+        Database db = catalog.getDbOrAnalysisException(tblName.getDb());
+        Table table = db.getTableOrAnalysisException(tblName.getTbl());
         table.readLock();
         try {
             if (partitions == null) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FromClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FromClause.java
index 5f09a14..83fab79 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FromClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FromClause.java
@@ -79,16 +79,9 @@ public class FromClause implements ParseNode, Iterable<TableRef> {
                 ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
             }
 
-            Database db = analyzer.getCatalog().getDb(dbName);
-            if (db == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-            }
-
+            Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
             String tblName = tableName.getTbl();
-            Table table = db.getTable(tblName);
-            if (table == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName);
-            }
+            Table table = db.getTableOrAnalysisException(tblName);
             if (table.getType() == Table.TableType.HIVE) {
                 throw new AnalysisException("Query from hive table is not supported, table: " + tblName);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
index a624cb0..c93efd8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
@@ -696,7 +696,7 @@ public class FunctionCallExpr extends Expr {
                             ConnectContext.get(), dbName, PrivPredicate.SELECT)) {
                         ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "SELECT");
                     }
-                    Database db = Catalog.getCurrentCatalog().getDb(dbName);
+                    Database db = Catalog.getCurrentCatalog().getDbNullable(dbName);
                     if (db != null) {
                         Function searchDesc = new Function(
                                 fnName, Arrays.asList(collectChildReturnTypes()), Type.INVALID, false);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java
index 64e3525..bc12383 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/InsertStmt.java
@@ -195,14 +195,8 @@ public class InsertStmt extends DdlStmt {
         String dbName = tblName.getDb();
         String tableName = tblName.getTbl();
         // check exist
-        Database db = analyzer.getCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-        Table table = db.getTable(tblName.getTbl());
-        if (table == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-        }
+        Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
+        Table table = db.getTableOrAnalysisException(tblName.getTbl());
 
         // check access
         if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, tableName,
@@ -299,7 +293,7 @@ public class InsertStmt extends DdlStmt {
         // create data sink
         createDataSink();
 
-        db = analyzer.getCatalog().getDb(tblName.getDb());
+        db = analyzer.getCatalog().getDbOrAnalysisException(tblName.getDb());
 
         // create label and begin transaction
         long timeoutSecond = ConnectContext.get().getSessionVariable().getQueryTimeoutS();
@@ -331,10 +325,7 @@ public class InsertStmt extends DdlStmt {
     private void analyzeTargetTable(Analyzer analyzer) throws AnalysisException {
         // Get table
         if (targetTable == null) {
-            targetTable = analyzer.getTable(tblName);
-            if (targetTable == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName.getTbl());
-            }
+            targetTable = analyzer.getTableOrAnalysisException(tblName);
         }
 
         if (targetTable instanceof OlapTable) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java
index 368d6c7..f68d58c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java
@@ -21,7 +21,6 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.OlapTable;
-import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
@@ -330,17 +329,12 @@ public class LoadStmt extends DdlStmt {
             if (dataDescription.isLoadFromTable()) {
                 isLoadFromTable = true;
             }
-            Database db = Catalog.getCurrentCatalog().getDb(label.getDbName());
-            if (db == null) {
-                throw new AnalysisException("database: " + label.getDbName() + "not  found.");
-            }
-            Table table = db.getTable(dataDescription.getTableName());
-            if (dataDescription.getMergeType() != LoadTask.MergeType.APPEND &&
-                    (!(table instanceof OlapTable) || ((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS)) {
+            Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(label.getDbName());
+            OlapTable table = db.getOlapTableOrAnalysisException(dataDescription.getTableName());
+            if (dataDescription.getMergeType() != LoadTask.MergeType.APPEND && table.getKeysType() != KeysType.UNIQUE_KEYS) {
                 throw new AnalysisException("load by MERGE or DELETE is only supported in unique tables.");
             }
-            if (dataDescription.getMergeType() != LoadTask.MergeType.APPEND
-                    && !((table instanceof OlapTable) && ((OlapTable) table).hasDeleteSign()) ) {
+            if (dataDescription.getMergeType() != LoadTask.MergeType.APPEND && !table.hasDeleteSign()) {
                 throw new AnalysisException("load by MERGE or DELETE need to upgrade table to support batch delete.");
             }
             if (brokerDesc != null && !brokerDesc.isMultiLoadBroker()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
index 146a57a..471e306 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
@@ -301,14 +301,8 @@ public class SelectStmt extends QueryStmt {
                 if (Strings.isNullOrEmpty(tableName)) {
                     ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR);
                 }
-                Database db = analyzer.getCatalog().getDb(dbName);
-                if (db == null) {
-                    ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-                }
-                Table table = db.getTable(tableName);
-                if (table == null) {
-                    ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-                }
+                Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
+                Table table = db.getTableOrAnalysisException(tableName);
 
                 // check auth
                 if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName,
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java
index ebed555..a50f396 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowAlterStmt.java
@@ -176,12 +176,8 @@ public class ShowAlterStmt extends ShowStmt {
     }
     
     
-    public void handleShowAlterTable(Analyzer analyzer) throws AnalysisException, UserException {
-        final String dbNameWithoutPrefix = ClusterNamespace.getNameFromFullName(dbName);
-        Database db = analyzer.getCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbNameWithoutPrefix);
-        }
+    public void handleShowAlterTable(Analyzer analyzer) throws UserException {
+        Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
 
         // build proc path
         StringBuilder sb = new StringBuilder();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java
index 1797db7..f02787d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDataStmt.java
@@ -106,10 +106,7 @@ public class ShowDataStmt extends ShowStmt {
             dbName = ClusterNamespace.getFullName(getClusterName(), dbName);
         }
         
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
 
         // order by
         if (orderByElements != null && !orderByElements.isEmpty()) {
@@ -231,7 +228,7 @@ public class ShowDataStmt extends ShowStmt {
                         tableName);
             }
 
-            OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableName, TableType.OLAP);
+            OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
             int i = 0;
             long totalSize = 0;
             long totalReplicaCount = 0;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java
index 954fe5c..1576b94 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowPartitionsStmt.java
@@ -109,11 +109,8 @@ public class ShowPartitionsStmt extends ShowStmt {
                                                 ConnectContext.get().getRemoteIP(),
                                                 tableName);
         }
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-        Table table = db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+        Table table = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         table.readLock();
         try {
             // build proc path
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java
index d43f008..4ead82b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowViewStmt.java
@@ -20,7 +20,6 @@ package org.apache.doris.analysis;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
-import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.View;
@@ -96,20 +95,8 @@ public class ShowViewStmt extends ShowStmt {
                     getTbl());
         }
 
-        Database database = Catalog.getCurrentCatalog().getDb(dbName);
-        if (database == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-
-        Table showTable = database.getTable(tbl.getTbl());
-        if (showTable == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, getTbl());
-        }
-
-        if (!(showTable instanceof OlapTable)) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, getTbl());
-        }
-
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+        database.getOlapTableOrAnalysisException(tbl.getTbl());
         for (Table table : database.getViews()) {
             View view = (View) table;
             List<TableRef> tblRefs = Lists.newArrayList();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/UpdateStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/UpdateStmt.java
index 4853b3f..63fb7d7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/UpdateStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/UpdateStmt.java
@@ -25,8 +25,6 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
 
 import com.google.common.base.Preconditions;
@@ -104,14 +102,8 @@ public class UpdateStmt extends DdlStmt {
         String targetTableName = tableName.getTbl();
         Preconditions.checkNotNull(dbName);
         Preconditions.checkNotNull(targetTableName);
-        Database database = Catalog.getCurrentCatalog().getDb(dbName);
-        if (database == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-        targetTable = database.getTable(tableName.getTbl());
-        if (targetTable == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName.getTbl());
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+        targetTable = database.getTableOrAnalysisException(tableName.getTbl());
         if (targetTable.getType() != Table.TableType.OLAP
                 || ((OlapTable) targetTable).getKeysType() != KeysType.UNIQUE_KEYS) {
             throw new AnalysisException("Only unique olap table could be updated.");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
index 5b18f2f..0571059 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
@@ -245,10 +245,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
 
         // check if db exist
         String dbName = stmt.getDbName();
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = catalog.getDbOrDdlException(dbName);
 
         // Try to get sequence lock.
         // We expect at most one operation on a repo at same time.
@@ -320,10 +317,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
         // Also calculate the signature for incremental backup check.
         for (TableRef tblRef : tblRefs) {
             String tblName = tblRef.getName().getTbl();
-            Table tbl = db.getTable(tblName);
-            if (tbl == null) {
-                ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName);
-            }
+            Table tbl = db.getTableOrDdlException(tblName);
             if (tbl.getType() == TableType.VIEW || tbl.getType() == TableType.ODBC) {
                 continue;
             }
@@ -544,10 +538,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
 
     public void cancel(CancelBackupStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = catalog.getDbOrDdlException(dbName);
         
         AbstractJob job = getCurrentJob(db.getId());
         if (job == null || (job instanceof BackupJob && stmt.isRestore())
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
index 384755c..d75e1c7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupJob.java
@@ -343,14 +343,11 @@ public class BackupJob extends AbstractJob {
 
     @Override
     public synchronized boolean isDone() {
-        if (state == BackupJobState.FINISHED || state == BackupJobState.CANCELLED) {
-            return true;
-        }
-        return false;
+        return state == BackupJobState.FINISHED || state == BackupJobState.CANCELLED;
     }
 
     private void prepareAndSendSnapshotTask() {
-        Database db = catalog.getDb(dbId);
+        Database db = catalog.getDbNullable(dbId);
         if (db == null) {
             status = new Status(ErrCode.NOT_FOUND, "database " + dbId + " does not exist");
             return;
@@ -364,7 +361,7 @@ public class BackupJob extends AbstractJob {
         AgentBatchTask batchTask = new AgentBatchTask();
         for (TableRef tableRef : tableRefs) {
             String tblName = tableRef.getName().getTbl();
-            Table tbl = db.getTable(tblName);
+            Table tbl = db.getTableNullable(tblName);
             if (tbl == null) {
                 status = new Status(ErrCode.NOT_FOUND, "table " + tblName + " does not exist");
                 return;
@@ -500,7 +497,7 @@ public class BackupJob extends AbstractJob {
         List<Resource> copiedResources = Lists.newArrayList();
         for (TableRef tableRef : tableRefs) {
             String tblName = tableRef.getName().getTbl();
-            Table table = db.getTable(tblName);
+            Table table = db.getTableNullable(tblName);
             table.readLock();
             try {
                 if (table.getType() == TableType.OLAP) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
index f368ca4..ab902d5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
@@ -52,6 +52,7 @@ import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.MarkedCountDownLatch;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.util.TimeUtils;
@@ -358,14 +359,14 @@ public class RestoreJob extends AbstractJob {
             return;
         }
 
-        Database db = catalog.getDb(dbId);
+        Database db = catalog.getDbNullable(dbId);
         if (db == null) {
             status = new Status(ErrCode.NOT_FOUND, "database " + dbId + " has been dropped");
             return;
         }
 
         for (IdChain idChain : fileMapping.getMapping().keySet()) {
-            OlapTable tbl = (OlapTable) db.getTable(idChain.getTblId());
+            OlapTable tbl = (OlapTable) db.getTableNullable(idChain.getTblId());
             if (tbl == null) {
                 status = new Status(ErrCode.NOT_FOUND, "table " + idChain.getTblId() + " has been dropped");
                 return;
@@ -413,7 +414,7 @@ public class RestoreJob extends AbstractJob {
      * 6. make snapshot for all replicas for incremental download later.
      */
     private void checkAndPrepareMeta() {
-        Database db = catalog.getDb(dbId);
+        Database db = catalog.getDbNullable(dbId);
         if (db == null) {
             status = new Status(ErrCode.NOT_FOUND, "database " + dbId + " does not exist");
             return;
@@ -431,7 +432,7 @@ public class RestoreJob extends AbstractJob {
         // Set all restored tbls' state to RESTORE
         // Table's origin state must be NORMAL and does not have unfinished load job.
         for (String tableName : jobInfo.backupOlapTableObjects.keySet()) {
-            Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tableName));
+            Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName));
             if (tbl == null) {
                 continue;
             }
@@ -470,7 +471,7 @@ public class RestoreJob extends AbstractJob {
         }
 
         for (BackupJobInfo.BackupViewInfo backupViewInfo : jobInfo.newBackupObjects.views) {
-            Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(backupViewInfo.name));
+            Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(backupViewInfo.name));
             if (tbl == null) {
                 continue;
             }
@@ -482,7 +483,7 @@ public class RestoreJob extends AbstractJob {
             }
         }
         for (BackupJobInfo.BackupOdbcTableInfo backupOdbcTableInfo : jobInfo.newBackupObjects.odbcTables) {
-            Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(backupOdbcTableInfo.dorisTableName));
+            Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(backupOdbcTableInfo.dorisTableName));
             if (tbl == null) {
                 continue;
             }
@@ -516,7 +517,7 @@ public class RestoreJob extends AbstractJob {
                 BackupOlapTableInfo tblInfo = olapTableEntry.getValue();
                 Table remoteTbl = backupMeta.getTable(tableName);
                 Preconditions.checkNotNull(remoteTbl);
-                Table localTbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tableName));
+                Table localTbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName));
                 if (localTbl != null) {
                     // table already exist, check schema
                     if (localTbl.getType() != TableType.OLAP) {
@@ -635,7 +636,7 @@ public class RestoreJob extends AbstractJob {
             // restore views
             for (BackupJobInfo.BackupViewInfo backupViewInfo : jobInfo.newBackupObjects.views) {
                 String backupViewName = backupViewInfo.name;
-                Table localTbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(backupViewName));
+                Table localTbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(backupViewName));
                 View remoteView = (View) backupMeta.getTable(backupViewName);
                 if (localTbl != null) {
                     Preconditions.checkState(localTbl.getType() == TableType.VIEW);
@@ -656,7 +657,7 @@ public class RestoreJob extends AbstractJob {
             // restore odbc external table
             for (BackupJobInfo.BackupOdbcTableInfo backupOdbcTableInfo : jobInfo.newBackupObjects.odbcTables) {
                 String backupOdbcTableName = backupOdbcTableInfo.dorisTableName;
-                Table localTbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(backupOdbcTableName));
+                Table localTbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(backupOdbcTableName));
                 OdbcTable remoteOdbcTable = (OdbcTable) backupMeta.getTable(backupOdbcTableName);
                 if (localTbl != null) {
                     Preconditions.checkState(localTbl.getType() == TableType.ODBC);
@@ -679,7 +680,7 @@ public class RestoreJob extends AbstractJob {
 
             // generate create replica tasks for all restored partitions
             for (Pair<String, Partition> entry : restoredPartitions) {
-                OlapTable localTbl = (OlapTable) db.getTable(entry.first);
+                OlapTable localTbl = (OlapTable) db.getTableNullable(entry.first);
                 Preconditions.checkNotNull(localTbl, localTbl.getName());
                 Partition restorePart = entry.second;
                 OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first);
@@ -745,14 +746,14 @@ public class RestoreJob extends AbstractJob {
         } else {
             ok = true;
         }
-            
+
         if (ok) {
             LOG.debug("finished to create all restored replcias. {}", this);
             // add restored partitions.
             // table should be in State RESTORE, so no other partitions can be
             // added to or removed from this table during the restore process.
             for (Pair<String, Partition> entry : restoredPartitions) {
-                OlapTable localTbl = (OlapTable) db.getTable(entry.first);
+                OlapTable localTbl = (OlapTable) db.getTableNullable(entry.first);
                 localTbl.writeLock();
                 try {
                     Partition restoredPart = entry.second;
@@ -823,7 +824,7 @@ public class RestoreJob extends AbstractJob {
         db.readLock();
         try {
             for (IdChain idChain : fileMapping.getMapping().keySet()) {
-                OlapTable tbl = (OlapTable) db.getTable(idChain.getTblId());
+                OlapTable tbl = (OlapTable) db.getTableNullable(idChain.getTblId());
                 tbl.readLock();
                 try {
                     Partition part = tbl.getPartition(idChain.getPartId());
@@ -1048,11 +1049,17 @@ public class RestoreJob extends AbstractJob {
     }
 
     private void replayCheckAndPrepareMeta() {
-        Database db = catalog.getDb(dbId);
+        Database db;
+        try {
+            db = catalog.getDbOrMetaException(dbId);
+        } catch (MetaNotFoundException e) {
+            LOG.warn("[INCONSISTENT META] replayCheckAndPrepareMeta failed", e);
+            return;
+        }
 
         // replay set all existing tables's state to RESTORE
         for (String tableName : jobInfo.backupOlapTableObjects.keySet()) {
-            Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tableName));
+            Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName));
             if (tbl == null) {
                 continue;
             }
@@ -1067,7 +1074,7 @@ public class RestoreJob extends AbstractJob {
 
         // restored partitions
         for (Pair<String, Partition> entry : restoredPartitions) {
-            OlapTable localTbl = (OlapTable) db.getTable(entry.first);
+            OlapTable localTbl = (OlapTable) db.getTableNullable(entry.first);
             Partition restorePart = entry.second;
             OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first);
             PartitionInfo localPartitionInfo = localTbl.getPartitionInfo();
@@ -1093,7 +1100,7 @@ public class RestoreJob extends AbstractJob {
                 }
             }
         }
-        
+
 
         // restored tables
         for (Table restoreTbl : restoredTbls) {
@@ -1167,7 +1174,7 @@ public class RestoreJob extends AbstractJob {
         for (long dbId : dbToSnapshotInfos.keySet()) {
             List<SnapshotInfo> infos = dbToSnapshotInfos.get(dbId);
 
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 status = new Status(ErrCode.NOT_FOUND, "db " + dbId + " does not exist");
                 return;
@@ -1206,7 +1213,7 @@ public class RestoreJob extends AbstractJob {
                         int currentBatchTaskNum = (batch == batchNum - 1) ? totalNum - index : taskNumPerBatch;
                         for (int j = 0; j < currentBatchTaskNum; j++) {
                             SnapshotInfo info = beSnapshotInfos.get(index++);
-                            Table tbl = db.getTable(info.getTblId());
+                            Table tbl = db.getTableNullable(info.getTblId());
                             if (tbl == null) {
                                 status = new Status(ErrCode.NOT_FOUND, "restored table "
                                         + info.getTabletId() + " does not exist");
@@ -1362,7 +1369,7 @@ public class RestoreJob extends AbstractJob {
     }
 
     private Status allTabletCommitted(boolean isReplay) {
-        Database db = catalog.getDb(dbId);
+        Database db = catalog.getDbNullable(dbId);
         if (db == null) {
             return new Status(ErrCode.NOT_FOUND, "database " + dbId + " does not exist");
         }
@@ -1371,7 +1378,7 @@ public class RestoreJob extends AbstractJob {
         // set all tables' state to NORMAL
         setTableStateToNormal(db);
         for (long tblId : restoredVersionInfo.rowKeySet()) {
-            Table tbl = db.getTable(tblId);
+            Table tbl = db.getTableNullable(tblId);
             if (tbl == null) {
                 continue;
             }
@@ -1425,7 +1432,7 @@ public class RestoreJob extends AbstractJob {
 
             catalog.getEditLog().logRestoreJob(this);
         }
-        
+
         LOG.info("job is finished. is replay: {}. {}", isReplay, this);
         return Status.OK;
     }
@@ -1530,7 +1537,7 @@ public class RestoreJob extends AbstractJob {
         }
 
         // clean restored objs
-        Database db = catalog.getDb(dbId);
+        Database db = catalog.getDbNullable(dbId);
         if (db != null) {
             // rollback table's state to NORMAL
             setTableStateToNormal(db);
@@ -1558,7 +1565,7 @@ public class RestoreJob extends AbstractJob {
 
             // remove restored partitions
             for (Pair<String, Partition> entry : restoredPartitions) {
-                OlapTable restoreTbl = (OlapTable) db.getTable(entry.first);
+                OlapTable restoreTbl = (OlapTable) db.getTableNullable(entry.first);
                 if (restoreTbl == null) {
                     continue;
                 }
@@ -1603,7 +1610,7 @@ public class RestoreJob extends AbstractJob {
 
     private void setTableStateToNormal(Database db) {
         for (String tableName : jobInfo.backupOlapTableObjects.keySet()) {
-            Table tbl = db.getTable(jobInfo.getAliasByOriginNameIfSet(tableName));
+            Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName));
             if (tbl == null) {
                 continue;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index 35a881a..a5d102a 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -248,6 +248,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import javax.annotation.Nullable;
 import java.io.BufferedReader;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -261,10 +262,10 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -426,7 +427,7 @@ public class Catalog {
     private SmallFileMgr smallFileMgr;
 
     private DynamicPartitionScheduler dynamicPartitionScheduler;
-    
+
     private PluginMgr pluginMgr;
 
     private AuditEventProcessor auditEventProcessor;
@@ -561,7 +562,7 @@ public class Catalog {
 
         this.metaContext = new MetaContext();
         this.metaContext.setThreadLocalInfo();
-        
+
         this.stat = new TabletSchedulerStat();
         this.tabletScheduler = new TabletScheduler(this, systemInfo, tabletInvertedIndex, stat, Config.tablet_rebalancer_type);
         this.tabletChecker = new TabletChecker(this, systemInfo, tabletScheduler, stat);
@@ -583,7 +584,7 @@ public class Catalog {
 
         this.dynamicPartitionScheduler = new DynamicPartitionScheduler("DynamicPartitionScheduler",
                 Config.dynamic_partition_check_interval_seconds * 1000L);
-        
+
         this.metaDir = Config.meta_dir;
         this.bdbDir = this.metaDir + BDB_DIR;
         this.imageDir = this.metaDir + IMAGE_DIR;
@@ -713,7 +714,7 @@ public class Catalog {
                             LOG.debug("catalog lock is held by: {}", Util.dumpThread(owner, 10));
                         }
                     }
-                    
+
                     if (mustLock) {
                         continue;
                     } else {
@@ -816,7 +817,7 @@ public class Catalog {
             LOG.info("wait catalog to be ready. FE type: {}. is ready: {}", feType, isReady.get());
         }
     }
-    
+
     public boolean isReady() {
         return isReady.get();
     }
@@ -1650,7 +1651,7 @@ public class Catalog {
                 Frontend fe = Frontend.read(dis);
                 replayAddFrontend(fe);
             }
-            
+
             size = dis.readInt();
             newChecksum ^= size;
             for (int i = 0; i < size; i++) {
@@ -1802,7 +1803,7 @@ public class Catalog {
             alterJobs.put(tableId, job);
 
             // init job
-            Database db = getDb(job.getDbId());
+            Database db = getDbNullable(job.getDbId());
             // should check job state here because the job is finished but not removed from alter jobs list
             if (db != null && (job.getState() == org.apache.doris.alter.AlterJob.JobState.PENDING
                     || job.getState() == org.apache.doris.alter.AlterJob.JobState.RUNNING)) {
@@ -2228,7 +2229,7 @@ public class Catalog {
         return checksum;
     }
 
-	public long saveResources(CountingDataOutputStream dos, long checksum) throws IOException {
+    public long saveResources(CountingDataOutputStream dos, long checksum) throws IOException {
         Catalog.getCurrentCatalog().getResourceMgr().write(dos);
         return checksum;
     }
@@ -2680,9 +2681,9 @@ public class Catalog {
             try {
                 if (!stmt.isForceDrop()) {
                     if (Catalog.getCurrentCatalog().getGlobalTransactionMgr().existCommittedTxns(db.getId(), null, null)) {
-                       throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed. " +
-                               "The database [" + dbName +"] cannot be dropped. If you want to forcibly drop(cannot be recovered)," +
-                               " please use \"DROP database FORCE\".");
+                        throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed. " +
+                                "The database [" + dbName + "] cannot be dropped. If you want to forcibly drop(cannot be recovered)," +
+                                " please use \"DROP database FORCE\".");
                     }
                 }
                 if (db.getDbState() == DbState.LINK && dbName.equals(db.getAttachDb())) {
@@ -2799,7 +2800,7 @@ public class Catalog {
 
     public void recoverDatabase(RecoverDbStmt recoverStmt) throws DdlException {
         // check is new db with same name already exist
-        if (getDb(recoverStmt.getDbName()) != null) {
+        if (getDb(recoverStmt.getDbName()).isPresent()) {
             throw new DdlException("Database[" + recoverStmt.getDbName() + "] already exist.");
         }
 
@@ -2833,20 +2834,14 @@ public class Catalog {
 
     public void recoverTable(RecoverTableStmt recoverStmt) throws DdlException {
         String dbName = recoverStmt.getDbName();
-
-        Database db = null;
-        if ((db = getDb(dbName)) == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-
         String tableName = recoverStmt.getTableName();
-        Table table = db.getTable(tableName);
-        if (table != null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName);
-        }
 
+        Database db = this.getDbOrDdlException(dbName);
         db.writeLock();
         try {
+            if (db.getTable(tableName).isPresent()) {
+                ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName);
+            }
             if (!Catalog.getCurrentRecycleBin().recoverTable(db, tableName)) {
                 ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
             }
@@ -2857,22 +2852,10 @@ public class Catalog {
 
     public void recoverPartition(RecoverPartitionStmt recoverStmt) throws DdlException {
         String dbName = recoverStmt.getDbName();
-
-        Database db = null;
-        if ((db = getDb(dbName)) == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-
         String tableName = recoverStmt.getTableName();
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-        }
 
-        if (table.getType() != TableType.OLAP) {
-            throw new DdlException("table[" + tableName + "] is not OLAP table");
-        }
-        OlapTable olapTable = (OlapTable) table;
+        Database db = this.getDbOrDdlException(dbName);
+        OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
         olapTable.writeLock();
         try {
             String partitionName = recoverStmt.getPartitionName();
@@ -2902,10 +2885,7 @@ public class Catalog {
 
     public void alterDatabaseQuota(AlterDatabaseQuotaStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
-        Database db = getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = this.getDbOrDdlException(dbName);
 
         QuotaType quotaType = stmt.getQuotaType();
         if (quotaType == QuotaType.DATA) {
@@ -2918,9 +2898,8 @@ public class Catalog {
         editLog.logAlterDb(dbInfo);
     }
 
-    public void replayAlterDatabaseQuota(String dbName, long quota, QuotaType quotaType) {
-        Database db = getDb(dbName);
-        Preconditions.checkNotNull(db);
+    public void replayAlterDatabaseQuota(String dbName, long quota, QuotaType quotaType) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(dbName);
         if (quotaType == QuotaType.DATA) {
             db.setDataQuotaWithLock(quota);
         } else if (quotaType == QuotaType.REPLICA) {
@@ -3014,16 +2993,13 @@ public class Catalog {
      * 10. add this table to FE's meta
      * 11. add this table to ColocateGroup if necessary
      */
-     public void createTable(CreateTableStmt stmt) throws DdlException {
+    public void createTable(CreateTableStmt stmt) throws DdlException {
         String engineName = stmt.getEngineName();
         String dbName = stmt.getDbName();
         String tableName = stmt.getTableName();
 
         // check if db exists
-        Database db = getDb(stmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = this.getDbOrDdlException(stmt.getDbName());
 
         // only internal table should check quota and cluster capacity
         if (!stmt.isExternal()) {
@@ -3034,7 +3010,7 @@ public class Catalog {
         }
 
         // check if table exists in db
-        if (db.getTable(tableName) != null) {
+        if (db.getTable(tableName).isPresent()) {
             if (stmt.isSetIfNotExists()) {
                 LOG.info("create table[{}] which already exists", tableName);
                 return;
@@ -3069,11 +3045,8 @@ public class Catalog {
 
     public void createTableLike(CreateTableLikeStmt stmt) throws DdlException {
         try {
-            Database db = Catalog.getCurrentCatalog().getDb(stmt.getExistedDbName());
-            Table table = db.getTable(stmt.getExistedTableName());
-            if (table == null) {
-                ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getExistedTableName());
-            }
+            Database db = Catalog.getCurrentCatalog().getDbOrDdlException(stmt.getExistedDbName());
+            Table table = db.getTableOrDdlException(stmt.getExistedTableName());
 
             if (table.getType() == TableType.VIEW) {
                 throw new DdlException("Not support create table from a View");
@@ -3114,26 +3087,15 @@ public class Catalog {
         DistributionDesc distributionDesc = addPartitionClause.getDistributionDesc();
         boolean isTempPartition = addPartitionClause.isTempPartition();
 
-        DistributionInfo distributionInfo = null;
-        OlapTable olapTable = null;
-
+        DistributionInfo distributionInfo;
         Map<Long, MaterializedIndexMeta> indexIdToMeta;
-        Set<String> bfColumns = null;
-
+        Set<String> bfColumns;
         String partitionName = singlePartitionDesc.getPartitionName();
 
         // check
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-        }
-
-        if (table.getType() != TableType.OLAP) {
-            throw new DdlException("Table[" + tableName + "] is not OLAP table");
-        }
-
+        Table table = db.getOlapTableOrDdlException(tableName);
         // check state
-        olapTable = (OlapTable) table;
+        OlapTable olapTable = (OlapTable) table;
 
         olapTable.readLock();
         try {
@@ -3240,17 +3202,10 @@ public class Catalog {
                     singlePartitionDesc.isInMemory(),
                     olapTable.getStorageFormat(),
                     singlePartitionDesc.getTabletType()
-                    );
+            );
 
             // check again
-            table = db.getTable(tableName);
-            if (table == null) {
-                ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-            }
-
-            if (table.getType() != TableType.OLAP) {
-                throw new DdlException("Table[" + tableName + "] is not OLAP table");
-            }
+            table = db.getOlapTableOrDdlException(tableName);
 
             table.writeLock();
             try {
@@ -3341,9 +3296,9 @@ public class Catalog {
         }
     }
 
-    public void replayAddPartition(PartitionPersistInfo info) throws UserException {
-        Database db = this.getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTableOrThrowException(info.getTableId(), TableType.OLAP);
+    public void replayAddPartition(PartitionPersistInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = info.getPartition();
@@ -3434,13 +3389,9 @@ public class Catalog {
         LOG.info("succeed in dropping partition[{}], is temp : {}, is force : {}", partitionName, isTempPartition, clause.isForceDrop());
     }
 
-    public void replayDropPartition(DropPartitionInfo info) {
-        Database db = this.getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
-        if (olapTable == null) {
-            LOG.warn("table {} does not exist when replaying drop rollup. db: {}", info.getTableId(), db.getId());
-            return;
-        }
+    public void replayDropPartition(DropPartitionInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             if (info.isTempPartition()) {
@@ -3457,19 +3408,14 @@ public class Catalog {
         Catalog.getCurrentRecycleBin().replayErasePartition(partitionId);
     }
 
-    public void replayRecoverPartition(RecoverInfo info) {
-        long dbId = info.getDbId();
-        Database db = getDb(dbId);
-        Table table = db.getTable(info.getTableId());
-        if (table == null) {
-            LOG.warn("table {} does not exist when replaying drop rollup. db: {}", info.getTableId(), db.getId());
-            return;
-        }
-        table.writeLock();
+    public void replayRecoverPartition(RecoverInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        olapTable.writeLock();
         try {
-            Catalog.getCurrentRecycleBin().replayRecoverPartition((OlapTable) table, info.getPartitionId());
+            Catalog.getCurrentRecycleBin().replayRecoverPartition(olapTable, info.getPartitionId());
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -3764,7 +3710,7 @@ public class Catalog {
                 shortKeyColumnCount, baseIndexStorageType, keysType);
 
         for (AlterClause alterClause : stmt.getRollupAlterClauseList()) {
-            AddRollupClause addRollupClause = (AddRollupClause)alterClause;
+            AddRollupClause addRollupClause = (AddRollupClause) alterClause;
 
             Long baseRollupIndex = olapTable.getIndexIdByName(tableName);
 
@@ -4408,9 +4354,9 @@ public class Catalog {
         }
     }
 
-    public void replayAlterExternalTableSchema(String dbName, String tableName, List<Column> newSchema) throws DdlException {
-        Database db = this.fullNameToDb.get(dbName);
-        Table table = db.getTable(tableName);
+    public void replayAlterExternalTableSchema(String dbName, String tableName, List<Column> newSchema) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(dbName);
+        Table table = db.getTableOrMetaException(tableName);
         table.writeLock();
         try {
             table.setNewFullSchema(newSchema);
@@ -4431,7 +4377,6 @@ public class Catalog {
             GroupId groupId = null;
             if (colocateIndex.isColocateTable(tabletMeta.getTableId())) {
                 // if this is a colocate table, try to get backend seqs from colocation index.
-                Database db = Catalog.getCurrentCatalog().getDb(tabletMeta.getDbId());
                 groupId = colocateIndex.getGroup(tabletMeta.getTableId());
                 backendsPerBucketSeq = colocateIndex.getBackendsPerBucketSeq(groupId);
             }
@@ -4466,7 +4411,7 @@ public class Catalog {
                     // get backends from existing backend sequence
                     chosenBackendIds = backendsPerBucketSeq.get(i);
                 }
-                
+
                 // create replicas
                 for (long backendId : chosenBackendIds) {
                     long replicaId = getNextId();
@@ -4474,7 +4419,7 @@ public class Catalog {
                             tabletMeta.getOldSchemaHash());
                     tablet.addReplica(replica);
                 }
-                Preconditions.checkState(chosenBackendIds.size() == replicationNum, chosenBackendIds.size() + " vs. "+ replicationNum);
+                Preconditions.checkState(chosenBackendIds.size() == replicationNum, chosenBackendIds.size() + " vs. " + replicationNum);
             }
 
             if (groupId != null && chooseBackendsArbitrary) {
@@ -4512,22 +4457,18 @@ public class Catalog {
         String tableName = stmt.getTableName();
 
         // check database
-        Database db = getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            if (stmt.isSetIfExists()) {
-                LOG.info("drop table[{}] which does not exist", tableName);
-                return;
-            } else {
-                ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-            }
-        }
+        Database db = this.getDbOrDdlException(dbName);
         db.writeLock();
         try {
+            Table table = db.getTableNullable(tableName);
+            if (table == null) {
+                if (stmt.isSetIfExists()) {
+                    LOG.info("drop table[{}] which does not exist", tableName);
+                    return;
+                } else {
+                    ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
+                }
+            }
             // Check if a view
             if (stmt.isView()) {
                 if (!(table instanceof View)) {
@@ -4542,7 +4483,7 @@ public class Catalog {
             if (!stmt.isForceDrop()) {
                 if (Catalog.getCurrentCatalog().getGlobalTransactionMgr().existCommittedTxns(db.getId(), table.getId(), null)) {
                     throw new DdlException("There are still some transactions in the COMMITTED state waiting to be completed. " +
-                            "The table [" + tableName +"] cannot be dropped. If you want to forcibly drop(cannot be recovered)," +
+                            "The table [" + tableName + "] cannot be dropped. If you want to forcibly drop(cannot be recovered)," +
                             " please use \"DROP table FORCE\".");
                 }
             }
@@ -4552,7 +4493,7 @@ public class Catalog {
                 if (table instanceof OlapTable && !stmt.isForceDrop()) {
                     OlapTable olapTable = (OlapTable) table;
                     if ((olapTable.getState() != OlapTableState.NORMAL)) {
-                        throw new DdlException("The table [" + tableName +"]'s state is " + olapTable.getState() + ", cannot be dropped." +
+                        throw new DdlException("The table [" + tableName + "]'s state is " + olapTable.getState() + ", cannot be dropped." +
                                 " please cancel the operation on olap table firstly. If you want to forcibly drop(cannot be recovered)," +
                                 " please use \"DROP table FORCE\".");
                     }
@@ -4590,16 +4531,8 @@ public class Catalog {
         return true;
     }
 
-    public void replayDropTable(Database db, long tableId, boolean isForceDrop) {
-        Table table = db.getTable(tableId);
-        // delete from db meta
-        if (table == null) {
-            /**
-             * Same as replayUpdateReplica()
-             */
-            LOG.warn("Olap table is null when the drop table log is replayed, tableId: {}", tableId);
-            return;
-        }
+    public void replayDropTable(Database db, long tableId, boolean isForceDrop) throws MetaNotFoundException {
+        Table table = db.getTableOrMetaException(tableId);
         db.writeLock();
         table.writeLock();
         try {
@@ -4610,13 +4543,12 @@ public class Catalog {
         }
     }
 
-    public void replayEraseTable(long tableId) throws DdlException {
+    public void replayEraseTable(long tableId) {
         Catalog.getCurrentRecycleBin().replayEraseTable(tableId);
     }
 
-    public void replayRecoverTable(RecoverInfo info) {
-        long dbId = info.getDbId();
-        Database db = getDb(dbId);
+    public void replayRecoverTable(RecoverInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
         db.writeLock();
         try {
             Catalog.getCurrentRecycleBin().replayRecoverTable(db, info.getTableId());
@@ -4625,10 +4557,8 @@ public class Catalog {
         }
     }
 
-    private void unprotectAddReplica(ReplicaPersistInfo info) {
+    private void unprotectAddReplica(OlapTable olapTable, ReplicaPersistInfo info) {
         LOG.debug("replay add a replica {}", info);
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
         Partition partition = olapTable.getPartition(info.getPartitionId());
         MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId());
         Tablet tablet = materializedIndex.getTablet(info.getTabletId());
@@ -4649,10 +4579,8 @@ public class Catalog {
         tablet.addReplica(replica);
     }
 
-    private void unprotectUpdateReplica(ReplicaPersistInfo info) {
+    private void unprotectUpdateReplica(OlapTable olapTable, ReplicaPersistInfo info) {
         LOG.debug("replay update a replica {}", info);
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
         Partition partition = olapTable.getPartition(info.getPartitionId());
         MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId());
         Tablet tablet = materializedIndex.getTablet(info.getTabletId());
@@ -4662,72 +4590,43 @@ public class Catalog {
         replica.setBad(false);
     }
 
-    public void replayAddReplica(ReplicaPersistInfo info) {
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
-        if (olapTable == null) {
-            /**
-             * Same as replayUpdateReplica()
-             */
-            LOG.warn("Olap table is null when the add replica log is replayed, {}", info);
-            return;
-        }
+    public void replayAddReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
-            unprotectAddReplica(info);
+            unprotectAddReplica(olapTable, info);
         } finally {
             olapTable.writeUnlock();
         }
     }
 
-    public void replayUpdateReplica(ReplicaPersistInfo info) {
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
-        if (olapTable == null) {
-            /**
-             * In the following cases, doris may record metadata modification information for a table that no longer exists.
-             * 1. Thread 1: get TableA object
-             * 2. Thread 2: lock db and drop table and record edit log of the dropped TableA
-             * 3. Thread 1: lock table, modify table and record edit log of the modified TableA
-             * **The modified edit log is after the dropped edit log**
-             * Because the table has been dropped, the olapTable in here is null when the modified edit log is replayed.
-             * So in this case, we will ignore the edit log of the modified table after the table is dropped.
-             */
-            LOG.warn("Olap table is null when the update replica log is replayed, {}", info);
-            return;
-        }
+    public void replayUpdateReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
-            unprotectUpdateReplica(info);
+            unprotectUpdateReplica(olapTable, info);
         } finally {
             olapTable.writeUnlock();
         }
     }
 
-    public void unprotectDeleteReplica(ReplicaPersistInfo info) {
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
+    public void unprotectDeleteReplica(OlapTable olapTable, ReplicaPersistInfo info) {
         Partition partition = olapTable.getPartition(info.getPartitionId());
         MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId());
         Tablet tablet = materializedIndex.getTablet(info.getTabletId());
         tablet.deleteReplicaByBackendId(info.getBackendId());
     }
 
-    public void replayDeleteReplica(ReplicaPersistInfo info) {
-        Database db = getDb(info.getDbId());
-        OlapTable tbl = (OlapTable) db.getTable(info.getTableId());
-        if (tbl == null) {
-            /**
-             * Same as replayUpdateReplica()
-             */
-            LOG.warn("Olap table is null when the delete replica log is replayed, {}", info);
-            return;
-        }
-        tbl.writeLock();
+    public void replayDeleteReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        olapTable.writeLock();
         try {
-            unprotectDeleteReplica(info);
+            unprotectDeleteReplica(olapTable, info);
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -4791,35 +4690,87 @@ public class Catalog {
         return token;
     }
 
-    public Database getDb(String name) {
-        if (fullNameToDb.containsKey(name)) {
-            return fullNameToDb.get(name);
+    @Nullable
+    public Database getDbNullable(String dbName) {
+        if (fullNameToDb.containsKey(dbName)) {
+            return fullNameToDb.get(dbName);
         } else {
             // This maybe a information_schema db request, and information_schema db name is case insensitive.
             // So, we first extract db name to check if it is information_schema.
             // Then we reassemble the origin cluster name with lower case db name,
             // and finally get information_schema db from the name map.
-            String dbName = ClusterNamespace.getNameFromFullName(name);
-            if (dbName.equalsIgnoreCase(InfoSchemaDb.DATABASE_NAME)) {
-                String clusterName = ClusterNamespace.getClusterNameFromFullName(name);
-                return fullNameToDb.get(ClusterNamespace.getFullName(clusterName, dbName.toLowerCase()));
+            String fullName = ClusterNamespace.getNameFromFullName(dbName);
+            if (fullName.equalsIgnoreCase(InfoSchemaDb.DATABASE_NAME)) {
+                String clusterName = ClusterNamespace.getClusterNameFromFullName(dbName);
+                fullName = ClusterNamespace.getFullName(clusterName, fullName.toLowerCase());
+                return fullNameToDb.get(fullName);
             }
         }
         return null;
     }
 
-    public Database getDb(long dbId) {
+    @Nullable
+    public Database getDbNullable(long dbId) {
         return idToDb.get(dbId);
     }
 
+    public Optional<Database> getDb(String dbName) {
+        return Optional.ofNullable(getDbNullable(dbName));
+    }
+
+    public Optional<Database> getDb(long dbId) {
+        return Optional.ofNullable(getDbNullable(dbId));
+    }
+
+    public <E extends Exception> Database
+    getDbOrException(String dbName, java.util.function.Function<String, E> e) throws E {
+        Database db = getDbNullable(dbName);
+        if (db == null) {
+            throw e.apply(dbName);
+        }
+        return db;
+    }
+
+    public <E extends Exception> Database
+    getDbOrException(long dbId, java.util.function.Function<Long, E> e) throws E {
+        Database db = getDbNullable(dbId);
+        if (db == null) {
+            throw e.apply(dbId);
+        }
+        return db;
+    }
+
+    public Database getDbOrMetaException(String dbName) throws MetaNotFoundException {
+        return getDbOrException(dbName, s -> new MetaNotFoundException("unknown databases, dbName=" + s));
+    }
+
+    public Database getDbOrMetaException(long dbId) throws MetaNotFoundException {
+        return getDbOrException(dbId, s -> new MetaNotFoundException("unknown databases, dbId=" + s));
+    }
+
+    public Database getDbOrDdlException(String dbName) throws DdlException {
+        return getDbOrException(dbName, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
+    }
+
+    public Database getDbOrDdlException(long dbId) throws DdlException {
+        return getDbOrException(dbId, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
+    }
+
+    public Database getDbOrAnalysisException(String dbName) throws AnalysisException {
+        return getDbOrException(dbName, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
+    }
+
+    public Database getDbOrAnalysisException(long dbId) throws AnalysisException {
+        return getDbOrException(dbId, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
+    }
+
     public EditLog getEditLog() {
         return editLog;
     }
 
     // Get the next available, needn't lock because of nextId is atomic.
     public long getNextId() {
-        long id = idGenerator.getNextId();
-        return id;
+        return idGenerator.getNextId();
     }
 
     public List<String> getDbNames() {
@@ -4848,7 +4799,7 @@ public class Catalog {
         List<Long> dbIds = getDbIds();
 
         for (long dbId : dbIds) {
-            Database db = getDb(dbId);
+            Database db = this.getDbNullable(dbId);
             if (db == null) {
                 LOG.warn("db {} does not exist while doing backend report", dbId);
                 continue;
@@ -4890,7 +4841,7 @@ public class Catalog {
 
         // handle data property changed
         for (Long dbId : changedPartitionsMap.keySet()) {
-            Database db = getDb(dbId);
+            Database db = getDbNullable(dbId);
             if (db == null) {
                 LOG.warn("db {} does not exist while checking backend storage medium", dbId);
                 continue;
@@ -4898,7 +4849,7 @@ public class Catalog {
             Multimap<Long, Long> tableIdToPartitionIds = changedPartitionsMap.get(dbId);
 
             for (Long tableId : tableIdToPartitionIds.keySet()) {
-                Table table = db.getTable(tableId);
+                Table table = db.getTableNullable(tableId);
                 if (table == null) {
                     continue;
                 }
@@ -5006,7 +4957,7 @@ public class Catalog {
         return sqlBlockRuleMgr;
     }
 
-    public RoutineLoadTaskScheduler getRoutineLoadTaskScheduler(){
+    public RoutineLoadTaskScheduler getRoutineLoadTaskScheduler() {
         return routineLoadTaskScheduler;
     }
 
@@ -5258,7 +5209,7 @@ public class Catalog {
         try {
             if (table instanceof OlapTable) {
                 OlapTable olapTable = (OlapTable) table;
-                if ( olapTable.getState() != OlapTableState.NORMAL) {
+                if (olapTable.getState() != OlapTableState.NORMAL) {
                     throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState());
                 }
             }
@@ -5270,7 +5221,7 @@ public class Catalog {
             }
 
             // check if name is already used
-            if (db.getTable(newTableName) != null) {
+            if (db.getTable(newTableName).isPresent()) {
                 throw new DdlException("Table name[" + newTableName + "] is already used");
             }
 
@@ -5300,15 +5251,15 @@ public class Catalog {
         LOG.info("refresh db[{}] table[{}] for schema change", db.getFullName(), table.getName());
     }
 
-    public void replayRenameTable(TableInfo tableInfo) throws DdlException {
+    public void replayRenameTable(TableInfo tableInfo) throws MetaNotFoundException {
         long dbId = tableInfo.getDbId();
         long tableId = tableInfo.getTableId();
         String newTableName = tableInfo.getNewTableName();
 
-        Database db = getDb(dbId);
+        Database db = this.getDbOrMetaException(dbId);
         db.writeLock();
         try {
-            Table table = db.getTable(tableId);
+            Table table = db.getTableOrMetaException(tableId);
             String tableName = table.getName();
             db.dropTable(tableName);
             table.writeLock();
@@ -5347,7 +5298,7 @@ public class Catalog {
                         } else if (bucketsNum != partition.getDistributionInfo().getBucketNum()) {
                             throw new DdlException("Partitions in table " + table.getName() + " have different buckets number");
                         }
-                        
+
                         if (replicationNum == -1) {
                             replicationNum = partitionInfo.getReplicationNum(partition.getId());
                         } else if (replicationNum != partitionInfo.getReplicationNum(partition.getId())) {
@@ -5359,7 +5310,7 @@ public class Catalog {
                 // set to an already exist colocate group, check if this table can be added to this group.
                 groupSchema.checkColocateSchema(table);
             }
-            
+
             List<List<Long>> backendsPerBucketSeq = null;
             if (groupSchema == null) {
                 // assign to a newly created group, set backends sequence.
@@ -5403,26 +5354,22 @@ public class Catalog {
                 table.getName(), isReplay);
     }
 
-    public void replayModifyTableColocate(TablePropertyInfo info) {
+    public void replayModifyTableColocate(TablePropertyInfo info) throws MetaNotFoundException {
+        long dbId = info.getGroupId().dbId;
         long tableId = info.getTableId();
         Map<String, String> properties = info.getPropertyMap();
 
-        Database db = getDb(info.getGroupId().dbId);
-        OlapTable table = (OlapTable) db.getTable(tableId);
-        if (table == null) {
-            LOG.warn("table {} does not exist when replaying modify table colocate. db: {}",
-                    tableId, info.getGroupId().dbId);
-            return;
-        }
-        table.writeLock();
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
         try {
-            modifyTableColocate(db, table, properties.get(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH), true,
+            modifyTableColocate(db, olapTable, properties.get(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH), true,
                     info.getGroupId());
         } catch (DdlException e) {
             // should not happen
             LOG.warn("failed to replay modify table colocate", e);
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -5466,28 +5413,24 @@ public class Catalog {
         }
     }
 
-    public void replayRenameRollup(TableInfo tableInfo) throws DdlException {
+    public void replayRenameRollup(TableInfo tableInfo) throws MetaNotFoundException {
         long dbId = tableInfo.getDbId();
         long tableId = tableInfo.getTableId();
         long indexId = tableInfo.getIndexId();
         String newRollupName = tableInfo.getNewRollupName();
 
-        Database db = getDb(dbId);
-        OlapTable table = (OlapTable) db.getTable(tableId);
-        if (table == null) {
-            LOG.warn("table {} does not exist when replaying rename rollup. db: {}", tableId, dbId);
-            return;
-        }
-        table.writeLock();
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
         try {
-            String rollupName = table.getIndexNameById(indexId);
-            Map<String, Long> indexNameToIdMap = table.getIndexNameToId();
+            String rollupName = olapTable.getIndexNameById(indexId);
+            Map<String, Long> indexNameToIdMap = olapTable.getIndexNameToId();
             indexNameToIdMap.remove(rollupName);
             indexNameToIdMap.put(newRollupName, indexId);
 
             LOG.info("replay rename rollup[{}] to {}", rollupName, newRollupName);
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -5531,25 +5474,21 @@ public class Catalog {
         }
     }
 
-    public void replayRenamePartition(TableInfo tableInfo) {
+    public void replayRenamePartition(TableInfo tableInfo) throws MetaNotFoundException {
         long dbId = tableInfo.getDbId();
         long tableId = tableInfo.getTableId();
         long partitionId = tableInfo.getPartitionId();
         String newPartitionName = tableInfo.getNewPartitionName();
 
-        Database db = getDb(dbId);
-        OlapTable table = (OlapTable) db.getTable(tableId);
-        if (table == null) {
-            LOG.warn("table {} does not exist when replaying rename partition. db: {}", tableId, dbId);
-            return;
-        }
-        table.writeLock();
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
         try {
-            Partition partition = table.getPartition(partitionId);
-            table.renamePartition(partition.getName(), newPartitionName);
+            Partition partition = olapTable.getPartition(partitionId);
+            olapTable.renamePartition(partition.getName(), newPartitionName);
             LOG.info("replay rename partition[{}] to {}", partition.getName(), newPartitionName);
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -5594,7 +5533,7 @@ public class Catalog {
     // The caller need to hold the table's write lock
     public void modifyTableReplicationNum(Database db, OlapTable table, Map<String, String> properties) throws DdlException {
         Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread());
-        String defaultReplicationNumName = "default."+ PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
+        String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
         PartitionInfo partitionInfo = table.getPartitionInfo();
         if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
             throw new DdlException("This is a partitioned table, you should specify partitions with MODIFY PARTITION clause." +
@@ -5655,7 +5594,7 @@ public class Catalog {
         tableProperty.buildInMemory();
 
         // need to update partition info meta
-        for(Partition partition: table.getPartitions()) {
+        for (Partition partition : table.getPartitions()) {
             table.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory());
         }
 
@@ -5663,17 +5602,13 @@ public class Catalog {
         editLog.logModifyInMemory(info);
     }
 
-    public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperationLog info) {
+    public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperationLog info) throws MetaNotFoundException {
         long dbId = info.getDbId();
         long tableId = info.getTableId();
         Map<String, String> properties = info.getProperties();
 
-        Database db = getDb(dbId);
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
-        if (olapTable == null) {
-            LOG.warn("table {} does not exist when replaying modify table property log. db: {}", tableId, dbId);
-            return;
-        }
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             TableProperty tableProperty = olapTable.getTableProperty();
@@ -5687,7 +5622,7 @@ public class Catalog {
 
             // need to replay partition info meta
             if (opCode == OperationType.OP_MODIFY_IN_MEMORY) {
-                for(Partition partition: olapTable.getPartitions()) {
+                for (Partition partition : olapTable.getPartitions()) {
                     olapTable.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory());
                 }
             }
@@ -5750,17 +5685,13 @@ public class Catalog {
         }
     }
 
-    public void replayModifyTableDefaultDistributionBucketNum(short opCode, ModifyTableDefaultDistributionBucketNumOperationLog info) {
+    public void replayModifyTableDefaultDistributionBucketNum(short opCode, ModifyTableDefaultDistributionBucketNumOperationLog info) throws MetaNotFoundException {
         long dbId = info.getDbId();
         long tableId = info.getTableId();
         int bucketNum = info.getBucketNum();
 
-        Database db = getDb(dbId);
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
-        if (olapTable == null) {
-            LOG.warn("table {} does not exist when replaying modify table default distribution bucket number log. db: {}", tableId, dbId);
-            return;
-        }
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             DistributionInfo defaultDistributionInfo = olapTable.getDefaultDistributionInfo();
@@ -5814,10 +5745,7 @@ public class Catalog {
             ErrorReport.reportDdlException(ErrorCode.ERR_DB_ACCESS_DENIED, ctx.getQualifiedUser(), qualifiedDb);
         }
 
-        if (getDb(qualifiedDb) == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, qualifiedDb);
-        }
-
+        this.getDbOrDdlException(qualifiedDb);
         ctx.setDatabase(qualifiedDb);
     }
 
@@ -5844,13 +5772,10 @@ public class Catalog {
         String tableName = stmt.getTable();
 
         // check if db exists
-        Database db = this.getDb(stmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = this.getDbOrDdlException(dbName);
 
         // check if table exists in db
-        if (db.getTable(tableName) != null) {
+        if (db.getTable(tableName).isPresent()) {
             if (stmt.isSetIfNotExists()) {
                 LOG.info("create view[{}] which already exists", tableName);
                 return;
@@ -5872,7 +5797,7 @@ public class Catalog {
         } catch (UserException e) {
             throw new DdlException("failed to init view stmt", e);
         }
-      
+
         if (!db.createTableWithLock(newView, false, stmt.isSetIfNotExists()).first) {
             throw new DdlException("Failed to create view[" + tableName + "].");
         }
@@ -5956,9 +5881,7 @@ public class Catalog {
     }
 
     private void unprotectCreateCluster(Cluster cluster) {
-        final Iterator<Long> iterator = cluster.getBackendIdList().iterator();
-        while (iterator.hasNext()) {
-            final Long id = iterator.next();
+        for (Long id : cluster.getBackendIdList()) {
             final Backend backend = systemInfo.getBackend(id);
             backend.setOwnerClusterName(cluster.getName());
             backend.setBackendState(BackendState.using);
@@ -6383,7 +6306,7 @@ public class Catalog {
                     final Set<String> tableNames = db.getTableNamesWithLock();
                     for (String tableName : tableNames) {
 
-                        Table table = db.getTable(tableName);
+                        Table table = db.getTableNullable(tableName);
                         if (table == null || table.getType() != TableType.OLAP) {
                             continue;
                         }
@@ -6455,12 +6378,12 @@ public class Catalog {
                 // for adding BE to some Cluster, but loadCluster is after loadBackend.
                 cluster.setBackendIdList(latestBackendIds);
 
-                String dbName =  InfoSchemaDb.getFullInfoSchemaDbName(cluster.getName());
+                String dbName = InfoSchemaDb.getFullInfoSchemaDbName(cluster.getName());
                 InfoSchemaDb db;
                 // Use real Catalog instance to avoid InfoSchemaDb id continuously increment
                 // when checkpoint thread load image.
                 if (Catalog.getServingCatalog().getFullNameToDb().containsKey(dbName)) {
-                    db = (InfoSchemaDb)Catalog.getServingCatalog().getFullNameToDb().get(dbName);
+                    db = (InfoSchemaDb) Catalog.getServingCatalog().getFullNameToDb().get(dbName);
                 } else {
                     db = new InfoSchemaDb(cluster.getName());
                     db.setClusterName(cluster.getName());
@@ -6600,8 +6523,7 @@ public class Catalog {
         try {
             // sort all dbs to avoid potential dead lock
             for (long dbId : getDbIds()) {
-                Database db = getDb(dbId);
-                Preconditions.checkNotNull(db);
+                Database db = this.getDbNullable(dbId);
                 databases.add(db);
             }
             databases.sort(Comparator.comparing(Database::getId));
@@ -6659,37 +6581,25 @@ public class Catalog {
 
         // check, and save some info which need to be checked again later
         Map<String, Long> origPartitions = Maps.newHashMap();
-        OlapTable copiedTbl = null;
-        Database db = getDb(dbTbl.getDb());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbTbl.getDb());
-        }
+        OlapTable copiedTbl;
 
         boolean truncateEntireTable = tblRef.getPartitionNames() == null;
 
-        Table table = db.getTable(dbTbl.getTbl());
-        if (table == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTbl.getTbl());
-        }
-
-        if (table.getType() != TableType.OLAP) {
-            throw new DdlException("Only support truncate OLAP table");
-        }
+        Database db = this.getDbOrDdlException(dbTbl.getDb());
+        OlapTable olapTable = db.getOlapTableOrDdlException(dbTbl.getTbl());
 
-        table.readLock();
+        olapTable.readLock();
         try {
-            OlapTable olapTable = (OlapTable) table;
             if (olapTable.getState() != OlapTableState.NORMAL) {
                 throw new DdlException("Table' state is not NORMAL: " + olapTable.getState());
             }
-            
+
             if (!truncateEntireTable) {
                 for (String partName : tblRef.getPartitionNames().getPartitionNames()) {
                     Partition partition = olapTable.getPartition(partName);
                     if (partition == null) {
                         throw new DdlException("Partition " + partName + " does not exist");
                     }
-                    
                     origPartitions.put(partName, partition.getId());
                 }
             } else {
@@ -6697,12 +6607,11 @@ public class Catalog {
                     origPartitions.put(partition.getName(), partition.getId());
                 }
             }
-            
             copiedTbl = olapTable.selectiveCopy(origPartitions.keySet(), true, IndexExtState.VISIBLE);
         } finally {
-            table.readUnlock();
+            olapTable.readUnlock();
         }
-        
+
         // 2. use the copied table to create partitions
         List<Partition> newPartitions = Lists.newArrayList();
         // tabletIdSet to save all newly created tablet ids.
@@ -6745,11 +6654,8 @@ public class Catalog {
         // all partitions are created successfully, try to replace the old partitions.
         // before replacing, we need to check again.
         // Things may be changed outside the table lock.
-        OlapTable olapTable = (OlapTable) db.getTable(copiedTbl.getId());
-        if (olapTable == null) {
-            throw new DdlException("Table[" + copiedTbl.getName() + "] is dropped");
-        }
-        table.writeLock();
+        olapTable = (OlapTable) db.getTableOrDdlException(copiedTbl.getId());
+        olapTable.writeLock();
         try {
             if (olapTable.getState() != OlapTableState.NORMAL) {
                 throw new DdlException("Table' state is not NORMAL: " + olapTable.getState());
@@ -6796,9 +6702,9 @@ public class Catalog {
                     truncateEntireTable);
             editLog.logTruncateTable(info);
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
-        
+
         LOG.info("finished to truncate table {}, partitions: {}",
                 tblRef.getName().toSql(), tblRef.getPartitionNames());
     }
@@ -6827,14 +6733,9 @@ public class Catalog {
         }
     }
 
-    public void replayTruncateTable(TruncateTableInfo info) {
-        Database db = getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTblId());
-        if (olapTable == null) {
-            LOG.warn("table {} does not exist when replaying truncate table log. db id: {}",
-                    info.getTblId(), info.getDbId());
-            return;
-        }
+    public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTblId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable());
@@ -6868,37 +6769,25 @@ public class Catalog {
 
     public void createFunction(CreateFunctionStmt stmt) throws UserException {
         FunctionName name = stmt.getFunctionName();
-        Database db = getDb(name.getDb());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, name.getDb());
-        }
+        Database db = this.getDbOrDdlException(name.getDb());
         db.addFunction(stmt.getFunction());
     }
 
-    public void replayCreateFunction(Function function) {
+    public void replayCreateFunction(Function function) throws MetaNotFoundException {
         String dbName = function.getFunctionName().getDb();
-        Database db = getDb(dbName);
-        if (db == null) {
-            throw new Error("unknown database when replay log, db=" + dbName);
-        }
+        Database db = this.getDbOrMetaException(dbName);
         db.replayAddFunction(function);
     }
 
     public void dropFunction(DropFunctionStmt stmt) throws UserException {
         FunctionName name = stmt.getFunctionName();
-        Database db = getDb(name.getDb());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, name.getDb());
-        }
+        Database db = this.getDbOrDdlException(name.getDb());
         db.dropFunction(stmt.getFunction());
     }
 
-    public void replayDropFunction(FunctionSearchDesc functionSearchDesc) {
+    public void replayDropFunction(FunctionSearchDesc functionSearchDesc) throws MetaNotFoundException {
         String dbName = functionSearchDesc.getName().getDb();
-        Database db = getDb(dbName);
-        if (db == null) {
-            throw new Error("unknown database when replay log, db=" + dbName);
-        }
+        Database db = this.getDbOrMetaException(dbName);
         db.replayDropFunction(functionSearchDesc);
     }
 
@@ -6931,19 +6820,17 @@ public class Catalog {
         // but we need to get replica from db->tbl->partition->...
         List<ReplicaPersistInfo> replicaPersistInfos = backendTabletsInfo.getReplicaPersistInfos();
         for (ReplicaPersistInfo info : replicaPersistInfos) {
-            long dbId = info.getDbId();
-            Database db = getDb(dbId);
-            if (db == null) {
-                continue;
-            }
-            OlapTable tbl = (OlapTable) db.getTable(info.getTableId());
-            if (tbl == null) {
+            OlapTable olapTable = (OlapTable) this.getDb(info.getDbId())
+                    .flatMap(db -> db.getTable(info.getTableId()))
+                    .filter(t -> t.getType() == TableType.OLAP)
+                    .orElse(null);
+            if (olapTable == null) {
                 continue;
             }
-            tbl.writeLock();
+            olapTable.writeLock();
             try {
 
-                Partition partition = tbl.getPartition(info.getPartitionId());
+                Partition partition = olapTable.getPartition(info.getPartitionId());
                 if (partition == null) {
                     continue;
                 }
@@ -6962,7 +6849,7 @@ public class Catalog {
                             info.getReplicaId(), info.getTabletId(), info.getBackendId());
                 }
             } finally {
-                tbl.writeUnlock();
+                olapTable.writeUnlock();
             }
         }
     }
@@ -6983,20 +6870,15 @@ public class Catalog {
         }
     }
 
-    public void replayConvertDistributionType(TableInfo tableInfo) {
-        Database db = getDb(tableInfo.getDbId());
-        OlapTable tbl = (OlapTable) db.getTable(tableInfo.getTableId());
-        if (tbl == null) {
-            LOG.warn("table {} does not exist when replaying convert distribution type. db: {}",
-                    tableInfo.getTableId(), tableInfo.getDbId());
-            return;
-        }
-        tbl.writeLock();
+    public void replayConvertDistributionType(TableInfo info) throws MetaNotFoundException {
+        Database db = this.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        olapTable.writeLock();
         try {
-            tbl.convertRandomDistributionToHashDistribution();
-            LOG.info("replay modify distribution type of table: " + tbl.getName());
+            olapTable.convertRandomDistributionToHashDistribution();
+            LOG.info("replay modify distribution type of table: " + olapTable.getName());
         } finally {
-            tbl.writeUnlock();
+            olapTable.writeUnlock();
         }
     }
 
@@ -7030,16 +6912,11 @@ public class Catalog {
                 clause.getPartitionNames(), clause.getTempPartitionNames(), olapTable.getName());
     }
 
-    public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog) {
-        Database db = getDb(replaceTempPartitionLog.getDbId());
-        if (db == null) {
-            return;
-        }
-        OlapTable olapTable = (OlapTable) db.getTable(replaceTempPartitionLog.getTblId());
-        if (olapTable == null) {
-            return;
-        }
-
+    public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog) throws MetaNotFoundException {
+        long dbId = replaceTempPartitionLog.getDbId();
+        long tableId = replaceTempPartitionLog.getTblId();
+        Database db = this.getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(),
@@ -7047,7 +6924,7 @@ public class Catalog {
                     replaceTempPartitionLog.isStrictRange(),
                     replaceTempPartitionLog.useTempPartitionName());
         } catch (DdlException e) {
-            LOG.warn("should not happen. {}", e);
+            throw new MetaNotFoundException(e);
         } finally {
             olapTable.writeUnlock();
         }
@@ -7070,11 +6947,11 @@ public class Catalog {
         return checksum;
     }
 
-    public void replayInstallPlugin(PluginInfo pluginInfo)  {
+    public void replayInstallPlugin(PluginInfo pluginInfo) throws MetaNotFoundException {
         try {
             pluginMgr.replayLoadDynamicPlugin(pluginInfo);
         } catch (Exception e) {
-            LOG.warn("replay install plugin failed.", e);
+            throw new MetaNotFoundException(e);
         }
     }
 
@@ -7086,11 +6963,11 @@ public class Catalog {
         LOG.info("uninstall plugin = " + stmt.getPluginName());
     }
 
-    public void replayUninstallPlugin(PluginInfo pluginInfo)  {
+    public void replayUninstallPlugin(PluginInfo pluginInfo) throws MetaNotFoundException {
         try {
             pluginMgr.uninstallPlugin(pluginInfo.getName());
         } catch (Exception e) {
-            LOG.warn("replay uninstall plugin failed.", e);
+            throw new MetaNotFoundException(e);
         }
     }
 
@@ -7114,47 +6991,39 @@ public class Catalog {
         setReplicaStatusInternal(tabletId, backendId, status, false);
     }
 
-    public void replaySetReplicaStatus(SetReplicaStatusOperationLog log) {
-        try {
-            setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true);
-        } catch (MetaNotFoundException e) {
-            LOG.warn("replay setReplicaStatus failed", e);
-        }
+    public void replaySetReplicaStatus(SetReplicaStatusOperationLog log) throws MetaNotFoundException {
+        setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true);
     }
 
     private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStatus status, boolean isReplay) throws MetaNotFoundException {
-        TabletMeta meta = tabletInvertedIndex.getTabletMeta(tabletId);
-        if (meta == null) {
-            throw new MetaNotFoundException(String.format("tablet %d does not exist", tabletId));
-        }
-        long dbId = meta.getDbId();
-        Database db = getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException(String.format("tablet %d in database %d does not exist", tabletId, dbId));
-        }
-        long tableId = meta.getTableId();
-        Table table = db.getTable(tableId);
-        if (table == null) {
-            throw new MetaNotFoundException(String.format("tablet %d of table %d in database %d does not exist", tabletId, tableId, dbId));
-        }
-        table.writeLock();
         try {
-            Replica replica = tabletInvertedIndex.getReplica(tabletId, backendId);
-            if (replica == null) {
-                throw new MetaNotFoundException(String.format("replica of tablet %d on backend %d does not exist", tabletId, backendId));
-            }
-            if (status == ReplicaStatus.BAD || status == ReplicaStatus.OK) {
-                if (replica.setBad(status == ReplicaStatus.BAD)) {
-                    if (!isReplay) {
-                        SetReplicaStatusOperationLog log = new SetReplicaStatusOperationLog(backendId, tabletId, status);
-                        getEditLog().logSetReplicaStatus(log);
+            TabletMeta meta = tabletInvertedIndex.getTabletMeta(tabletId);
+            if (meta == null) {
+                throw new MetaNotFoundException("tablet does not exist");
+            }
+            Database db = this.getDbOrMetaException(meta.getDbId());
+            Table table = db.getTableOrMetaException(meta.getTableId());
+            table.writeLock();
+            try {
+                Replica replica = tabletInvertedIndex.getReplica(tabletId, backendId);
+                if (replica == null) {
+                    throw new MetaNotFoundException("replica does not exist on backend, beId=" + backendId);
+                }
+                if (status == ReplicaStatus.BAD || status == ReplicaStatus.OK) {
+                    if (replica.setBad(status == ReplicaStatus.BAD)) {
+                        if (!isReplay) {
+                            SetReplicaStatusOperationLog log = new SetReplicaStatusOperationLog(backendId, tabletId, status);
+                            getEditLog().logSetReplicaStatus(log);
+                        }
+                        LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}",
+                                replica.getId(), tabletId, backendId, status, isReplay);
                     }
-                    LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}",
-                            replica.getId(), tabletId, backendId, status, isReplay);
                 }
+            } finally {
+                table.writeUnlock();
             }
-        } finally {
-            table.writeUnlock();
+        } catch (MetaNotFoundException e) {
+            throw new MetaNotFoundException("set replica status failed, tabletId=" + tabletId, e);
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
index bdee431..333a89e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
@@ -540,7 +540,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             // we need to get olap table to get schema hash info
             // first find it in catalog. if not found, it should be in recycle bin
             OlapTable olapTable = null;
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 // just log. db should be in recycle bin
                 if (!idToDatabase.containsKey(dbId)) {
@@ -550,7 +550,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
                     continue;
                 }
             } else {
-                olapTable = (OlapTable) db.getTable(tableId);
+                olapTable = (OlapTable) db.getTableNullable(tableId);
             }
 
             if (olapTable == null) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
index 756d268..68c552d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
@@ -18,6 +18,7 @@
 package org.apache.doris.catalog;
 
 import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.persist.ColocatePersistInfo;
@@ -441,16 +442,9 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    public void replayAddTableToGroup(ColocatePersistInfo info) {
-        Database db = Catalog.getCurrentCatalog().getDb(info.getGroupId().dbId);
-        Preconditions.checkNotNull(db);
-        OlapTable tbl = (OlapTable) db.getTable(info.getTableId());
-        if (tbl == null) {
-            LOG.warn("table {} does not exist when replaying rename rollup. db: {}",
-                    info.getTableId(), info.getGroupId().dbId);
-            return;
-        }
-
+    public void replayAddTableToGroup(ColocatePersistInfo info) throws MetaNotFoundException {
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getGroupId().dbId);
+        OlapTable tbl = db.getTableOrMetaException(info.getTableId(), Table.TableType.OLAP);
         writeLock();
         try {
             if (!group2BackendsPerBucketSeq.containsKey(info.getGroupId())) {
@@ -655,14 +649,14 @@ public class ColocateTableIndex implements Writable {
 
         for (Map.Entry<Long, Long> entry : tmpGroup2Db.entrySet()) {
             GroupId groupId = new GroupId(entry.getValue(), entry.getKey());
-            Database db = Catalog.getCurrentCatalog().getDb(groupId.dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(groupId.dbId);
             if (db == null) {
                 continue;
             }
             Collection<Long> tableIds = tmpGroup2Tables.get(groupId.grpId);
 
             for (Long tblId : tableIds) {
-                OlapTable tbl = (OlapTable) db.getTable(tblId);
+                OlapTable tbl = (OlapTable) db.getTableNullable(tblId);
                 if (tbl == null) {
                     continue;
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
index c3ad35a..7c4db1e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
@@ -22,6 +22,7 @@ import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.MetaNotFoundException;
@@ -42,6 +43,7 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import javax.annotation.Nullable;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -51,6 +53,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
@@ -360,7 +363,7 @@ public class Database extends MetaObject implements Writable {
         if (Catalog.isStoredTableNamesLowerCase()) {
             tableName = tableName.toLowerCase();
         }
-        Table table = getTable(tableName);
+        Table table = getTableNullable(tableName);
         if (table != null) {
             this.nameToTable.remove(tableName);
             this.idToTable.remove(table.getId());
@@ -415,11 +418,9 @@ public class Database extends MetaObject implements Writable {
 
     /**
      * This is a thread-safe method when nameToTable is a concurrent hash map
-     *
-     * @param tableName
-     * @return
      */
-    public Table getTable(String tableName) {
+    @Nullable
+    public Table getTableNullable(String tableName) {
         if (Catalog.isStoredTableNamesLowerCase()) {
             tableName = tableName.toLowerCase();
         }
@@ -433,50 +434,92 @@ public class Database extends MetaObject implements Writable {
     }
 
     /**
-     * This is a thread-safe method when nameToTable is a concurrent hash map
-     *
-     * @param tableName
-     * @param tableType
-     * @return
+     * This is a thread-safe method when idToTable is a concurrent hash map
      */
-    public Table getTableOrThrowException(String tableName, TableType tableType) throws MetaNotFoundException {
-        Table table = getTable(tableName);
+    @Nullable
+    public Table getTableNullable(long tableId) {
+        return idToTable.get(tableId);
+    }
+
+    public Optional<Table> getTable(String tableName) {
+        return Optional.ofNullable(getTableNullable(tableName));
+    }
+
+    public Optional<Table> getTable(long tableId) {
+        return Optional.ofNullable(getTableNullable(tableId));
+    }
+
+    public <E extends Exception> Table getTableOrException(String tableName, java.util.function.Function<String, E> e) throws E {
+        Table table = getTableNullable(tableName);
         if (table == null) {
-            throw new MetaNotFoundException("unknown table, table=" + tableName);
-        }
-        if (table.getType() != tableType) {
-            throw new MetaNotFoundException("table type is not " + tableType + ", table=" + tableName + ", type=" + table.getClass());
+            throw e.apply(tableName);
         }
         return table;
     }
 
-    /**
-     * This is a thread-safe method when idToTable is a concurrent hash map
-     *
-     * @param tableId
-     * @return
-     */
-    public Table getTable(long tableId) {
-        return idToTable.get(tableId);
+    public <E extends Exception> Table getTableOrException(long tableId, java.util.function.Function<Long, E> e) throws E {
+        Table table = getTableNullable(tableId);
+        if (table == null) {
+            throw e.apply(tableId);
+        }
+        return table;
     }
 
+    public Table getTableOrMetaException(String tableName) throws MetaNotFoundException {
+        return getTableOrException(tableName, t -> new MetaNotFoundException("unknown table, tableName=" + t));
+    }
 
-    /**
-     * This is a thread-safe method when idToTable is a concurrent hash map
-     *
-     * @param tableId
-     * @param tableType
-     * @return
-     */
-    public Table getTableOrThrowException(long tableId, TableType tableType) throws MetaNotFoundException {
-        Table table = idToTable.get(tableId);
-        if (table == null) {
-            throw new MetaNotFoundException("unknown table, tableId=" + tableId);
+    public Table getTableOrMetaException(long tableId) throws MetaNotFoundException {
+        return getTableOrException(tableId, t -> new MetaNotFoundException("unknown table, tableId=" + t));
+    }
+    @SuppressWarnings("unchecked")
+    public <T extends Table> T getTableOrMetaException(String tableName, TableType tableType) throws MetaNotFoundException {
+        Table table = getTableOrMetaException(tableName);
+        if (table.getType() != tableType) {
+            throw new MetaNotFoundException("table type is not " + tableType + ", tableName=" + tableName + ", type=" + table.getType());
         }
+        return (T) table;
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T extends Table> T getTableOrMetaException(long tableId, TableType tableType) throws MetaNotFoundException {
+        Table table = getTableOrMetaException(tableId);
         if (table.getType() != tableType) {
-            throw new MetaNotFoundException("table type is not " + tableType + ", tableId=" + tableId + ", type=" + table.getClass());
+            throw new MetaNotFoundException("table type is not " + tableType + ", tableId=" + tableId + ", type=" + table.getType());
         }
-        return table;
+        return (T) table;
+    }
+
+    public Table getTableOrDdlException(String tableName) throws DdlException {
+        return getTableOrException(tableName, t -> new DdlException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t)));
+    }
+
+    public OlapTable getOlapTableOrDdlException(String tableName) throws DdlException {
+        Table table = getTableOrDdlException(tableName);
+        if (!(table instanceof OlapTable)) {
+            throw new DdlException(ErrorCode.ERR_NOT_OLAP_TABLE.formatErrorMsg(tableName));
+        }
+        return (OlapTable) table;
+    }
+
+    public Table getTableOrDdlException(long tableId) throws DdlException {
+        return getTableOrException(tableId, t -> new DdlException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t)));
+    }
+
+    public Table getTableOrAnalysisException(String tableName) throws AnalysisException {
+        return getTableOrException(tableName, t -> new AnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t)));
+    }
+
+    public OlapTable getOlapTableOrAnalysisException(String tableName) throws AnalysisException {
+        Table table = getTableOrAnalysisException(tableName);
+        if (!(table instanceof OlapTable)) {
+            throw new AnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE.formatErrorMsg(tableName));
+        }
+        return (OlapTable) table;
+    }
+
+    public Table getTableOrAnalysisException(long tableId) throws AnalysisException {
+        return getTableOrException(tableId, t -> new AnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t)));
     }
 
     public int getMaxReplicationNum() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java
index b555f98..cede16e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EncryptKeyHelper.java
@@ -20,8 +20,7 @@ package org.apache.doris.catalog;
 import org.apache.doris.analysis.CreateEncryptKeyStmt;
 import org.apache.doris.analysis.DropEncryptKeyStmt;
 import org.apache.doris.analysis.EncryptKeyName;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -32,37 +31,25 @@ public class EncryptKeyHelper {
 
     public static void createEncryptKey(CreateEncryptKeyStmt stmt) throws UserException {
         EncryptKeyName name = stmt.getEncryptKeyName();
-        Database db = Catalog.getCurrentCatalog().getDb(name.getDb());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, name.getDb());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(name.getDb());
         db.addEncryptKey(stmt.getEncryptKey());
     }
 
-    public static void replayCreateEncryptKey(EncryptKey encryptKey) {
+    public static void replayCreateEncryptKey(EncryptKey encryptKey) throws MetaNotFoundException {
         String dbName = encryptKey.getEncryptKeyName().getDb();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new Error("unknown database when replay log, db=" + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbName);
         db.replayAddEncryptKey(encryptKey);
     }
 
     public static void dropEncryptKey(DropEncryptKeyStmt stmt) throws UserException {
         EncryptKeyName name = stmt.getEncryptKeyName();
-        Database db = Catalog.getCurrentCatalog().getDb(name.getDb());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, name.getDb());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(name.getDb());
         db.dropEncryptKey(stmt.getEncryptKeysSearchDesc());
     }
 
-    public static void replayDropEncryptKey(EncryptKeySearchDesc encryptKeySearchDesc) {
+    public static void replayDropEncryptKey(EncryptKeySearchDesc encryptKeySearchDesc) throws MetaNotFoundException {
         String dbName = encryptKeySearchDesc.getKeyEncryptKeyName().getDb();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new Error("unknown database when replay log, db=" + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbName);
         db.replayDropEncryptKey(encryptKeySearchDesc);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java
index 78666f6..b17c265 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/InfoSchemaDb.java
@@ -71,8 +71,8 @@ public class InfoSchemaDb extends Database {
     }
 
     @Override
-    public Table getTable(String name) {
-        return super.getTable(name.toLowerCase());
+    public Table getTableNullable(String name) {
+        return super.getTableNullable(name.toLowerCase());
     }
 
     public static String getFullInfoSchemaDbName(String cluster) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
index feecd2a..8734264 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
@@ -24,7 +24,6 @@ import org.apache.doris.analysis.BinaryPredicate.Operator;
 import org.apache.doris.analysis.PartitionNames;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.Replica.ReplicaStatus;
-import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.system.Backend;
@@ -52,20 +51,11 @@ public class MetadataViewer {
         Catalog catalog = Catalog.getCurrentCatalog();
         SystemInfoService infoService = Catalog.getCurrentSystemInfo();
 
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-
-        Table tbl = db.getTable(tblName);
-        if (tbl == null || tbl.getType() != TableType.OLAP) {
-            throw new DdlException("Table does not exist or is not OLAP table: " + tblName);
-        }
+        Database db = catalog.getDbOrDdlException(dbName);
+        OlapTable olapTable = db.getOlapTableOrDdlException(tblName);
 
-        tbl.readLock();
+        olapTable.readLock();
         try {
-            OlapTable olapTable = (OlapTable) tbl;
-            
             if (partitions.isEmpty()) {
                 partitions.addAll(olapTable.getPartitionNames());
             } else {
@@ -147,7 +137,7 @@ public class MetadataViewer {
                 }
             }
         } finally {
-            tbl.readUnlock();
+            olapTable.readUnlock();
         }
 
         return result;
@@ -177,19 +167,10 @@ public class MetadataViewer {
         Catalog catalog = Catalog.getCurrentCatalog();
         SystemInfoService infoService = Catalog.getCurrentSystemInfo();
 
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-
-        Table tbl = db.getTable(tblName);
-        if (tbl == null || tbl.getType() != TableType.OLAP) {
-            throw new DdlException("Table does not exist or is not OLAP table: " + tblName);
-        }
-
-        tbl.readLock();
+        Database db = catalog.getDbOrDdlException(dbName);
+        OlapTable olapTable = db.getOlapTableOrDdlException(tblName);
+        olapTable.readLock();
         try {
-            OlapTable olapTable = (OlapTable) tbl;
             List<Long> partitionIds = Lists.newArrayList();
             if (partitionNames == null) {
                 for (Partition partition : olapTable.getPartitions()) {
@@ -251,7 +232,7 @@ public class MetadataViewer {
             }
             
         } finally {
-            tbl.readUnlock();
+            olapTable.readUnlock();
         }
 
         return result;
@@ -282,19 +263,11 @@ public class MetadataViewer {
             throw new DdlException("Should specify one and only one partitions");
         }
 
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-
-        Table tbl = db.getTable(tblName);
-        if (tbl == null || tbl.getType() != TableType.OLAP) {
-            throw new DdlException("Table does not exist or is not OLAP table: " + tblName);
-        }
+        Database db = catalog.getDbOrDdlException(dbName);
+        OlapTable olapTable = db.getOlapTableOrDdlException(tblName);
 
-        tbl.readLock();
+        olapTable.readLock();
         try {
-            OlapTable olapTable = (OlapTable) tbl;
             long partitionId = -1;
             // check partition
             for (String partName : partitionNames.getPartitionNames()) {
@@ -344,7 +317,7 @@ public class MetadataViewer {
                 result.add(row);
             }
         } finally {
-            tbl.readUnlock();
+            olapTable.readUnlock();
         }
 
         return result;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java
index 898a97a..ed02c39 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java
@@ -82,7 +82,7 @@ public class TabletStatMgr extends MasterDaemon {
         start = System.currentTimeMillis();
         List<Long> dbIds = Catalog.getCurrentCatalog().getDbIds();
         for (Long dbId : dbIds) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
index d566eb0..00bc1a1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
@@ -140,7 +140,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
         // get all groups
         Set<GroupId> groupIds = colocateIndex.getAllGroupIds();
         for (GroupId groupId : groupIds) {
-            Database db = catalog.getDb(groupId.dbId);
+            Database db = catalog.getDbNullable(groupId.dbId);
             if (db == null) {
                 continue;
             }
@@ -184,7 +184,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
         Set<GroupId> groupIds = colocateIndex.getAllGroupIds();
         for (GroupId groupId : groupIds) {
             List<Long> tableIds = colocateIndex.getAllTableIds(groupId);
-            Database db = catalog.getDb(groupId.dbId);
+            Database db = catalog.getDbNullable(groupId.dbId);
             if (db == null) {
                 continue;
             }
@@ -196,7 +196,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
 
             boolean isGroupStable = true;
             OUT: for (Long tableId : tableIds) {
-                OlapTable olapTable = (OlapTable) db.getTable(tableId);
+                OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
                 if (olapTable == null || !colocateIndex.isColocateTable(olapTable.getId())) {
                     continue;
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
index 0162bc7..e75624f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
@@ -302,7 +302,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             Pair<Long, Long> tableInfo = iterator.next();
             Long dbId = tableInfo.first;
             Long tableId = tableInfo.second;
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 iterator.remove();
                 continue;
@@ -313,7 +313,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             String tableName;
             boolean skipAddPartition = false;
             OlapTable olapTable;
-            olapTable = (OlapTable) db.getTable(tableId);
+            olapTable = (OlapTable) db.getTableNullable(tableId);
             // Only OlapTable has DynamicPartitionProperty
             if (olapTable == null
                     || !olapTable.dynamicPartitionExists()
@@ -409,7 +409,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
 
     private void initDynamicPartitionTable() {
         for (Long dbId : Catalog.getCurrentCatalog().getDbIds()) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
index 77238c4..f6bd08d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
@@ -27,7 +27,6 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Partition.PartitionState;
 import org.apache.doris.catalog.Table;
-import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.Tablet.TabletStatus;
 import org.apache.doris.clone.TabletScheduler.AddResult;
@@ -242,14 +241,14 @@ public class TabletChecker extends MasterDaemon {
 
         OUT:
         for (long dbId : copiedPrios.rowKeySet()) {
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
             List<Long> aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true);
             Map<Long, Set<PrioPart>> tblPartMap = copiedPrios.row(dbId);
             for (long tblId : tblPartMap.keySet()) {
-                OlapTable tbl = (OlapTable) db.getTable(tblId);
+                OlapTable tbl = (OlapTable) db.getTableNullable(tblId);
                 if (tbl == null) {
                     continue;
                 }
@@ -277,7 +276,7 @@ public class TabletChecker extends MasterDaemon {
         List<Long> dbIds = catalog.getDbIds();
         OUT:
         for (Long dbId : dbIds) {
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -437,7 +436,7 @@ public class TabletChecker extends MasterDaemon {
         while (iter.hasNext()) {
             Map.Entry<Long, Map<Long, Set<PrioPart>>> dbEntry = iter.next();
             long dbId = dbEntry.getKey();
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 iter.remove();
                 continue;
@@ -447,7 +446,7 @@ public class TabletChecker extends MasterDaemon {
             while (jter.hasNext()) {
                 Map.Entry<Long, Set<PrioPart>> tblEntry = jter.next();
                 long tblId = tblEntry.getKey();
-                OlapTable tbl = (OlapTable) db.getTable(tblId);
+                OlapTable tbl = (OlapTable) db.getTableNullable(tblId);
                 if (tbl == null) {
                     deletedPrios.add(Pair.create(dbId, tblId));
                     continue;
@@ -525,22 +524,15 @@ public class TabletChecker extends MasterDaemon {
 
     public static RepairTabletInfo getRepairTabletInfo(String dbName, String tblName, List<String> partitions) throws DdlException {
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
+        Database db = catalog.getDbOrDdlException(dbName);
 
         long dbId = db.getId();
         long tblId = -1;
         List<Long> partIds = Lists.newArrayList();
-        Table tbl = db.getTable(tblName);
-        if (tbl == null || tbl.getType() != TableType.OLAP) {
-            throw new DdlException("Table does not exist or is not OLAP table: " + tblName);
-        }
-        tbl.readLock();
+        OlapTable olapTable = db.getOlapTableOrDdlException(tblName);
+        olapTable.readLock();
         try {
-            tblId = tbl.getId();
-            OlapTable olapTable = (OlapTable) tbl;
+            tblId = olapTable.getId();
 
             if (partitions == null || partitions.isEmpty()) {
                 partIds = olapTable.getPartitions().stream().map(Partition::getId).collect(Collectors.toList());
@@ -554,7 +546,7 @@ public class TabletChecker extends MasterDaemon {
                 }
             }
         } finally {
-            tbl.readUnlock();
+            olapTable.readUnlock();
         }
 
         Preconditions.checkState(tblId != -1);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
index 8e3ab51..d1ffbc1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
@@ -637,9 +637,9 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
             AgentTaskQueue.removeTask(cloneTask.getBackendId(), TTaskType.CLONE, cloneTask.getSignature());
 
             // clear all CLONE replicas
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db != null) {
-                Table table = db.getTable(tblId);
+                Table table = db.getTableNullable(tblId);
                 if (table != null) {
                     table.writeLock();
                     try {
@@ -795,14 +795,8 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
         }
 
         // 1. check the tablet status first
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "db does not exist");
-        }
-        OlapTable olapTable = (OlapTable) db.getTable(tblId);
-        if (olapTable == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new SchedException(Status.UNRECOVERABLE, "db does not exist"));
+        OlapTable olapTable = (OlapTable) db.getTableOrException(tblId, s -> new SchedException(Status.UNRECOVERABLE, "tbl does not exist"));
 
         olapTable.writeLock();
         try {
@@ -810,23 +804,23 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
             if (partition == null) {
                 throw new SchedException(Status.UNRECOVERABLE, "partition does not exist");
             }
-            
+
             MaterializedIndex index = partition.getIndex(indexId);
             if (index == null) {
                 throw new SchedException(Status.UNRECOVERABLE, "index does not exist");
             }
-            
+
             if (schemaHash != olapTable.getSchemaHashByIndexId(indexId)) {
                 throw new SchedException(Status.UNRECOVERABLE, "schema hash is not consistent. index's: "
                         + olapTable.getSchemaHashByIndexId(indexId)
                         + ", task's: " + schemaHash);
             }
-            
+
             Tablet tablet = index.getTablet(tabletId);
             if (tablet == null) {
                 throw new SchedException(Status.UNRECOVERABLE, "tablet does not exist");
             }
-            
+
             List<Long> aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true);
             short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId);
             Pair<TabletStatus, TabletSchedCtx.Priority> pair = tablet.getHealthStatusWithPriority(
@@ -835,9 +829,9 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
             if (pair.first == TabletStatus.HEALTHY) {
                 throw new SchedException(Status.FINISHED, "tablet is healthy");
             }
-            
+
             // tablet is unhealthy, go on
-            
+
             // Here we do not check if the clone version is equal to the partition's visible version.
             // Because in case of high frequency loading, clone version always lags behind the visible version,
             // But we will check if the clone replica's version is larger than or equal to the task's visible version.
@@ -850,20 +844,20 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
                         visibleVersion, visibleVersionHash);
                 throw new SchedException(Status.RUNNING_FAILED, msg);
             }
-            
+
             // check if replica exist
             Replica replica = tablet.getReplicaByBackendId(destBackendId);
             if (replica == null) {
                 throw new SchedException(Status.UNRECOVERABLE,
                         "replica does not exist. backend id: " + destBackendId);
             }
-            
+
             replica.updateVersionInfo(reportedTablet.getVersion(), reportedTablet.getVersionHash(),
                     reportedTablet.getDataSize(), reportedTablet.getRowCount());
             if (reportedTablet.isSetPathHash()) {
                 replica.setPathHash(reportedTablet.getPathHash());
             }
-            
+
             if (this.type == Type.BALANCE) {
                 long partitionVisibleVersion = partition.getVisibleVersion();
                 if (replica.getVersion() < partitionVisibleVersion) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
index 9c4b2b4..56aee10 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
@@ -451,17 +451,10 @@ public class TabletScheduler extends MasterDaemon {
         tabletCtx.setLastVisitedTime(currentTime);
         stat.counterTabletScheduled.incrementAndGet();
 
-        // check this tablet again
-        Database db = catalog.getDb(tabletCtx.getDbId());
-        if (db == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "db does not exist");
-        }
-
         Pair<TabletStatus, TabletSchedCtx.Priority> statusPair;
-        OlapTable tbl = (OlapTable) db.getTable(tabletCtx.getTblId());
-        if (tbl == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist");
-        }
+        // check this tablet again
+        Database db = catalog.getDbOrException(tabletCtx.getDbId(), s -> new SchedException(Status.UNRECOVERABLE, "db does not exist"));
+        OlapTable tbl = (OlapTable) db.getTableOrException(tabletCtx.getTblId(), s -> new SchedException(Status.UNRECOVERABLE, "tbl does not exist"));
         tbl.writeLock();
         try {
             boolean isColocateTable = colocateTableIndex.isColocateTable(tbl.getId());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/MetaNotFoundException.java b/fe/fe-core/src/main/java/org/apache/doris/common/MetaNotFoundException.java
index 9fbd6b7..14b5ed9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/MetaNotFoundException.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/MetaNotFoundException.java
@@ -29,6 +29,10 @@ public class MetaNotFoundException extends UserException {
         super(errcode, msg);
     }
 
+    public MetaNotFoundException(Throwable e) {
+        super(e);
+    }
+
     public MetaNotFoundException(String msg, Throwable e) {
         super(msg, e);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/DbsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/DbsProcDir.java
index 9734a31..08419d4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/DbsProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/DbsProcDir.java
@@ -68,10 +68,7 @@ public class DbsProcDir implements ProcDirInterface {
             throw new AnalysisException("Invalid db id format: " + dbIdStr);
         }
 
-        Database db = catalog.getDb(dbId);
-        if (db == null) {
-            throw new AnalysisException("Database[" + dbId + "] does not exist.");
-        }
+        Database db = catalog.getDbOrAnalysisException(dbId);
 
         return new TablesProcDir(db);
     }
@@ -91,7 +88,7 @@ public class DbsProcDir implements ProcDirInterface {
         // get info
         List<List<Comparable>> dbInfos = new ArrayList<List<Comparable>>();
         for (String dbName : dbNames) {
-            Database db = catalog.getDb(dbName);
+            Database db = catalog.getDbNullable(dbName);
             if (db == null) {
                 continue;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/JobsDbProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/JobsDbProcDir.java
index c9da8e3..ba1dc88 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/JobsDbProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/JobsDbProcDir.java
@@ -60,10 +60,7 @@ public class JobsDbProcDir implements ProcDirInterface {
             throw new AnalysisException("Invalid db id format: " + dbIdStr);
         }
 
-        Database db = catalog.getDb(dbId);
-        if (db == null) {
-            throw new AnalysisException("Database[" + dbId + "] does not exist.");
-        }
+        Database db = catalog.getDbOrAnalysisException(dbId);
 
         return new JobsProcDir(catalog, db);
     }
@@ -77,13 +74,11 @@ public class JobsDbProcDir implements ProcDirInterface {
         result.setNames(TITLE_NAMES);
         List<String> names = catalog.getDbNames();
         if (names == null || names.isEmpty()) {
-            // empty
             return result;
         }
 
         for (String name : names) {
-            Database db = catalog.getDb(name);
-            result.addRow(Lists.newArrayList(String.valueOf(db.getId()), name));
+            catalog.getDb(name).ifPresent(db -> result.addRow(Lists.newArrayList(String.valueOf(db.getId()), name)));
         }
 
         return result;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
index 596267c..6f051bc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
@@ -105,7 +105,7 @@ public class StatisticProcDir implements ProcDirInterface {
                 // skip information_schema database
                 continue;
             }
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java
index 380618d..58f079a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TablesProcDir.java
@@ -65,17 +65,14 @@ public class TablesProcDir implements ProcDirInterface {
             throw new AnalysisException("TableIdStr is null");
         }
 
-        long tableId = -1L;
+        long tableId;
         try {
-            tableId = Long.valueOf(tableIdStr);
+            tableId = Long.parseLong(tableIdStr);
         } catch (NumberFormatException e) {
             throw new AnalysisException("Invalid table id format: " + tableIdStr);
         }
 
-        Table table = db.getTable(tableId);
-        if (table == null) {
-            throw new AnalysisException("Table[" + tableId + "] does not exist");
-        }
+        Table table = db.getTableOrAnalysisException(tableId);
 
         return new TableProcDir(db, table);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java
index 3bafb68..7ee6354 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/SmallFileMgr.java
@@ -165,20 +165,14 @@ public class SmallFileMgr implements Writable {
 
     public void createFile(CreateFileStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         downloadAndAddFile(db.getId(), stmt.getCatalogName(), stmt.getFileName(),
                 stmt.getDownloadUrl(), stmt.getChecksum(), stmt.isSaveContent());
     }
 
     public void dropFile(DropFileStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         removeFile(db.getId(), stmt.getCatalogName(), stmt.getFileName(), false);
     }
 
@@ -459,11 +453,7 @@ public class SmallFileMgr implements Writable {
     }
 
     public List<List<String>> getInfo(String dbName) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-        
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         List<List<String>> infos = Lists.newArrayList();
         synchronized (files) {
             if (files.containsRow(db.getId())) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
index aa183b1..a10e7e4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
@@ -115,7 +115,7 @@ public class CheckConsistencyJob {
             return false;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(tabletMeta.getDbId());
+        Database db = Catalog.getCurrentCatalog().getDbNullable(tabletMeta.getDbId());
         if (db == null) {
             LOG.debug("db[{}] does not exist", tabletMeta.getDbId());
             return false;
@@ -130,7 +130,7 @@ public class CheckConsistencyJob {
         Tablet tablet = null;
 
         AgentBatchTask batchTask = new AgentBatchTask();
-        Table table = db.getTable(tabletMeta.getTableId());
+        Table table = db.getTableNullable(tabletMeta.getTableId());
         if (table == null) {
             LOG.debug("table[{}] does not exist", tabletMeta.getTableId());
             return false;
@@ -253,14 +253,14 @@ public class CheckConsistencyJob {
             return -1;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(tabletMeta.getDbId());
+        Database db = Catalog.getCurrentCatalog().getDbNullable(tabletMeta.getDbId());
         if (db == null) {
             LOG.warn("db[{}] does not exist", tabletMeta.getDbId());
             return -1;
         }
 
         boolean isConsistent = true;
-        Table table = db.getTable(tabletMeta.getTableId());
+        Table table = db.getTableNullable(tabletMeta.getTableId());
         if (table == null) {
             LOG.warn("table[{}] does not exist", tabletMeta.getTableId());
             return -1;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
index 052847b..acab74d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
@@ -28,6 +28,7 @@ import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.common.util.TimeUtils;
 import org.apache.doris.consistency.CheckConsistencyJob.JobState;
@@ -248,7 +249,7 @@ public class ConsistencyChecker extends MasterDaemon {
                 // skip 'information_schema' database
                 continue;
             }
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -362,9 +363,9 @@ public class ConsistencyChecker extends MasterDaemon {
         job.handleFinishedReplica(backendId, checksum);
     }
 
-    public void replayFinishConsistencyCheck(ConsistencyCheckInfo info, Catalog catalog) {
-        Database db = catalog.getDb(info.getDbId());
-        OlapTable table = (OlapTable) db.getTable(info.getTableId());
+    public void replayFinishConsistencyCheck(ConsistencyCheckInfo info, Catalog catalog) throws MetaNotFoundException {
+        Database db = catalog.getDbOrMetaException(info.getDbId());
+        OlapTable table = (OlapTable) db.getTableOrMetaException(info.getTableId());
         table.writeLock();
         try {
             Partition partition = table.getPartition(info.getPartitionId());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRepository.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRepository.java
index 2d94ee8..9581641 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRepository.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRepository.java
@@ -91,7 +91,10 @@ public class EsRepository extends MasterDaemon {
         }
         List<Long> dbIds = Catalog.getCurrentCatalog().getDbIds();
         for (Long dbId : dbIds) {
-            Database database = Catalog.getCurrentCatalog().getDb(dbId);
+            Database database = Catalog.getCurrentCatalog().getDbNullable(dbId);
+            if (database == null) {
+                continue;
+            }
 
             List<Table> tables = database.getTables();
             for (Table table : tables) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/CancelStreamLoad.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/CancelStreamLoad.java
index ec0c37d..23f12f4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/CancelStreamLoad.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/CancelStreamLoad.java
@@ -70,10 +70,7 @@ public class CancelStreamLoad extends RestBaseAction {
         // FIXME(cmy)
         // checkWritePriv(authInfo.fullUserName, fullDbName);
 
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new DdlException("unknown database, database=" + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(fullDbName);
 
         try {
             Catalog.getCurrentGlobalTransactionMgr().abortTransaction(db.getId(), label, "user cancel");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java
index 61c2631..e9c2342 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetDdlStmtAction.java
@@ -72,20 +72,13 @@ public class GetDdlStmtAction extends RestBaseAction {
             throw new DdlException("Missing params. Need database name and Table name");
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
+        Table table = db.getTableOrDdlException(tableName);
 
         List<String> createTableStmt = Lists.newArrayList();
         List<String> addPartitionStmt = Lists.newArrayList();
         List<String> createRollupStmt = Lists.newArrayList();
 
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            throw new DdlException("Table[" + tableName + "] does not exist");
-        }
-
         table.readLock();
         try {
             Catalog.getDdlStmt(table, createTableStmt, addPartitionStmt, createRollupStmt, true, false /* show password */);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetStreamLoadState.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetStreamLoadState.java
index 57b2348..6be1236 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetStreamLoadState.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/GetStreamLoadState.java
@@ -70,10 +70,7 @@ public class GetStreamLoadState extends RestBaseAction {
         // FIXME(cmy)
         // checkReadPriv(authInfo.fullUserName, fullDbName);
 
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new DdlException("unknown database, database=" + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(fullDbName);
 
         String state = Catalog.getCurrentGlobalTransactionMgr().getLabelState(db.getId(), label).toString();
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java
index 50bbdd6..6f36c57 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/MigrationAction.java
@@ -77,26 +77,14 @@ public class MigrationAction extends RestBaseAction {
             throw new DdlException("Missing params. Need database name");
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         List<List<Comparable>> rows = Lists.newArrayList();
 
 
 
         if (!Strings.isNullOrEmpty(tableName)) {
-            Table table = db.getTable(tableName);
-            if (table == null) {
-                throw new DdlException("Table[" + tableName + "] does not exist");
-            }
-
-            if (table.getType() != TableType.OLAP) {
-                throw new DdlException("Table[" + tableName + "] is not OlapTable");
-            }
-
-            OlapTable olapTable = (OlapTable) table;
+            OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
             olapTable.readLock();
             try {
                 for (Partition partition : olapTable.getPartitions()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java
index c9b9f98..e134fc3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/RowCountAction.java
@@ -24,8 +24,6 @@ import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Replica;
-import org.apache.doris.catalog.Table;
-import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.http.ActionController;
@@ -74,23 +72,10 @@ public class RowCountAction extends RestBaseAction {
 
         Map<String, Long> indexRowCountMap = Maps.newHashMap();
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
-
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            throw new DdlException("Table[" + tableName + "] does not exist");
-        }
-
-        if (table.getType() != TableType.OLAP) {
-            throw new DdlException("Table[" + tableName + "] is not OLAP table");
-        }
-
-        table.writeLock();
+        Database db = catalog.getDbOrDdlException(dbName);
+        OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
+        olapTable.writeLock();
         try {
-            OlapTable olapTable = (OlapTable) table;
             for (Partition partition : olapTable.getAllPartitions()) {
                 long version = partition.getVisibleVersion();
                 long versionHash = partition.getVisibleVersionHash();
@@ -111,7 +96,7 @@ public class RowCountAction extends RestBaseAction {
                 } // end for indices
             } // end for partitions            
         } finally {
-            table.writeUnlock();
+            olapTable.writeUnlock();
         }
 
         // to json response
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java
index faf99f3..e3bbef3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/ShowMetaInfoAction.java
@@ -159,14 +159,15 @@ public class ShowMetaInfoAction extends RestBaseAction {
         Map<String, Long> result = new HashMap<String, Long>();
         List<String> dbNames = Catalog.getCurrentCatalog().getDbNames();
 
-        for (int i = 0; i < dbNames.size(); i++) {
-            String dbName = dbNames.get(i);
-            Database db = Catalog.getCurrentCatalog().getDb(dbName);
+        for (String dbName : dbNames) {
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbName);
+            if (db == null) {
+                continue;
+            }
 
             long totalSize = 0;
             List<Table> tables = db.getTables();
-            for (int j = 0; j < tables.size(); j++) {
-                Table table = tables.get(j);
+            for (Table table : tables) {
                 if (table.getType() != TableType.OLAP) {
                     continue;
                 }
@@ -195,7 +196,7 @@ public class ShowMetaInfoAction extends RestBaseAction {
                 } // end for partitions
                 totalSize += tableSize;
             } // end for tables
-            result.put(dbName, Long.valueOf(totalSize));
+            result.put(dbName, totalSize);
         } // end for dbs
         return result;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java
index b1906c5..f9968d2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/StorageTypeCheckAction.java
@@ -61,10 +61,7 @@ public class StorageTypeCheckAction extends RestBaseAction {
         }
 
         String fullDbName = ClusterNamespace.getFullName(ConnectContext.get().getClusterName(), dbName);
-        Database db = catalog.getDb(fullDbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
+        Database db = catalog.getDbOrDdlException(fullDbName);
 
         JSONObject root = new JSONObject();
         List<Table> tableList = db.getTables();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java
index 27042d7..7634fab 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableQueryPlanAction.java
@@ -28,6 +28,7 @@ import org.apache.doris.catalog.Table;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.DorisHttpException;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.http.ActionController;
 import org.apache.doris.http.BaseRequest;
 import org.apache.doris.http.BaseResponse;
@@ -126,22 +127,12 @@ public class TableQueryPlanAction extends RestBaseAction {
             String fullDbName = ClusterNamespace.getFullName(ConnectContext.get().getClusterName(), dbName);
             // check privilege for select, otherwise return HTTP 401
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tableName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND,
-                                             "Database [" + dbName + "] " + "does not exists");
-            }
-            Table table = db.getTable(tableName);
-            if (table == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND,
-                        "Table [" + tableName + "] " + "does not exists");
-            }
-            // just only support OlapTable, ignore others such as ESTable
-            if (table.getType() != Table.TableType.OLAP) {
-                // Forbidden
-                throw new DorisHttpException(HttpResponseStatus.FORBIDDEN,
-                        "only support OlapTable currently, but Table [" + tableName + "] "
-                                + "is not a OlapTable");
+            Table table;
+            try {
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                table = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+            } catch (MetaNotFoundException e) {
+                throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, e.getMessage());
             }
 
             // may be should acquire writeLock
@@ -163,8 +154,7 @@ public class TableQueryPlanAction extends RestBaseAction {
             // send result with extra information
             response.setContentType("application/json");
             response.getContent().append(result);
-            sendResult(request, response,
-                       HttpResponseStatus.valueOf(Integer.parseInt(String.valueOf(resultMap.get("status")))));
+            sendResult(request, response, HttpResponseStatus.valueOf(Integer.parseInt(String.valueOf(resultMap.get("status")))));
         } catch (Exception e) {
             // may be this never happen
             response.getContent().append(e.getMessage());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java
index b255625..3b75547 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableRowCountAction.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.Table;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.DorisHttpException;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.http.ActionController;
 import org.apache.doris.http.BaseRequest;
 import org.apache.doris.http.BaseResponse;
@@ -75,20 +76,12 @@ public class TableRowCountAction extends RestBaseAction {
             String fullDbName = ClusterNamespace.getFullName(ConnectContext.get().getClusterName(), dbName);
             // check privilege for select, otherwise return HTTP 401
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tableName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Database [" + dbName + "] " + "does not exists");
-            }
-
-            Table table = db.getTable(tableName);
-            if (table == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists");
-            }
-            // just only support OlapTable, ignore others such as ESTable
-            if (!(table instanceof OlapTable)) {
-                // Forbidden
-                throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] "
-                        + "is not a OlapTable, only support OlapTable currently");
+            Table table;
+            try {
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                table = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+            } catch (MetaNotFoundException e) {
+                throw new DorisHttpException(HttpResponseStatus.BAD_REQUEST, e.getMessage());
             }
 
             table.writeLock();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java
index c986870..e298042 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/TableSchemaAction.java
@@ -20,7 +20,6 @@ package org.apache.doris.http.rest;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
-import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Table;
@@ -28,6 +27,7 @@ import org.apache.doris.catalog.Type;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.DorisHttpException;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.http.ActionController;
 import org.apache.doris.http.BaseRequest;
 import org.apache.doris.http.BaseResponse;
@@ -77,20 +77,12 @@ public class TableSchemaAction extends RestBaseAction {
             String fullDbName = ClusterNamespace.getFullName(ConnectContext.get().getClusterName(), dbName);
             // check privilege for select, otherwise return 401 HTTP status
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tableName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Database [" + dbName + "] " + "does not exists");
-            }
-
-            Table table = db.getTable(tableName);
-            if (table == null) {
-                throw new DorisHttpException(HttpResponseStatus.NOT_FOUND, "Table [" + tableName + "] " + "does not exists");
-            }
-            // just only support OlapTable, ignore others such as ESTable
-            if (!(table instanceof OlapTable)) {
-                // Forbidden
-                throw new DorisHttpException(HttpResponseStatus.FORBIDDEN, "Table [" + tableName + "] "
-                        + "is not a OlapTable, only support OlapTable currently");
+            Table table;
+            try {
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                table = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+            } catch (MetaNotFoundException e) {
+                throw new DorisHttpException(HttpResponseStatus.BAD_REQUEST, e.getMessage());
             }
 
             table.readLock();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CancelLoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CancelLoadAction.java
index ff5f971..4b716b4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CancelLoadAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/CancelLoadAction.java
@@ -19,6 +19,7 @@ package org.apache.doris.httpv2.rest;
 
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 import org.apache.doris.httpv2.exception.UnauthorizedException;
@@ -63,9 +64,11 @@ public class CancelLoadAction extends RestBaseController {
             return ResponseEntityBuilder.badRequest("No label specified");
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("unknown database, database=" + dbName);
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         // TODO(cmy): Currently we only check priv in db level.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java
index 45ac72b..ce6d93a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetDdlStmtAction.java
@@ -20,6 +20,7 @@ package org.apache.doris.httpv2.rest;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
@@ -63,20 +64,18 @@ public class GetDdlStmtAction extends RestBaseController {
         }
 
         String fullDbName = getFullDbName(dbName);
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database[" + dbName + "] does not exist");
+        Table table;
+        try {
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+            table = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         List<String> createTableStmt = Lists.newArrayList();
         List<String> addPartitionStmt = Lists.newArrayList();
         List<String> createRollupStmt = Lists.newArrayList();
 
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            return ResponseEntityBuilder.okWithCommonError("Table[" + tableName + "] does not exist");
-        }
-
         table.readLock();
         try {
             Catalog.getDdlStmt(table, createTableStmt, addPartitionStmt, createRollupStmt, true, false /* show password */);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java
index 194b63f..208008a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/GetStreamLoadState.java
@@ -19,6 +19,7 @@ package org.apache.doris.httpv2.rest;
 
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 
 import com.google.common.base.Strings;
@@ -52,9 +53,11 @@ public class GetStreamLoadState extends RestBaseController {
 
         final String fullDbName = getFullDbName(dbName);
 
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("unknown database, database=" + dbName);
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         String state = Catalog.getCurrentGlobalTransactionMgr().getLabelState(db.getId(), label).toString();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java
index aedc469..e00d54d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/MetaInfoAction.java
@@ -25,6 +25,7 @@ import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.proc.ProcNodeInterface;
@@ -140,9 +141,11 @@ public class MetaInfoAction extends RestBaseController {
 
 
         String fullDbName = getFullDbName(dbName);
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database does not exist: " + fullDbName);
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         List<String> tblNames = Lists.newArrayList();
@@ -215,19 +218,18 @@ public class MetaInfoAction extends RestBaseController {
         String fullDbName = getFullDbName(dbName);
         checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tblName, PrivPredicate.SHOW);
 
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database does not exist: " + fullDbName);
-        }
-
         String withMvPara = request.getParameter(PARAM_WITH_MV);
-        boolean withMv = Strings.isNullOrEmpty(withMvPara) ? false : withMvPara.equals("1");
+        boolean withMv = !Strings.isNullOrEmpty(withMvPara) && withMvPara.equals("1");
 
         // get all proc paths
         Map<String, Map<String, Object>> result = Maps.newHashMap();
-        Table tbl = db.getTable(tblName);
-        if (tbl == null) {
-            return ResponseEntityBuilder.okWithCommonError("Table does not exist: " + tblName);
+        Database db;
+        Table tbl;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+            tbl = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
         tbl.readLock();
         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java
index bfe7563..c4f7d16 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RowCountAction.java
@@ -66,14 +66,10 @@ public class RowCountAction extends RestBaseController {
 
         String fullDbName = getFullDbName(dbName);
         Map<String, Long> indexRowCountMap = Maps.newHashMap();
-        Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database[" + fullDbName + "] does not exist");
-        }
-        OlapTable olapTable = null;
+        OlapTable olapTable;
         try {
-            olapTable = (OlapTable) db.getTableOrThrowException(tableName, Table.TableType.OLAP);
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+            olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java
index d98e248..e08ebe5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/ShowAction.java
@@ -277,11 +277,11 @@ public class ShowAction extends RestBaseController {
         Map<String, Long> result = new HashMap<String, Long>();
         List<String> dbNames = Catalog.getCurrentCatalog().getDbNames();
 
-        for (int i = 0; i < dbNames.size(); i++) {
-            String dbName = dbNames.get(i);
-            Database db = Catalog.getCurrentCatalog().getDb(dbName);
-            long totalSize = getDataSizeOfDatabase(db);
-            result.put(dbName, Long.valueOf(totalSize));
+        for (String dbName : dbNames) {
+            Catalog.getCurrentCatalog().getDb(dbName).ifPresent(db -> {
+                long totalSize = getDataSizeOfDatabase(db);
+                result.put(dbName, totalSize);
+            });
         } // end for dbs
         return result;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java
index cf35a8e..3845a2e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StorageTypeCheckAction.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Table.TableType;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.qe.ConnectContext;
@@ -54,9 +55,11 @@ public class StorageTypeCheckAction extends RestBaseController {
         }
 
         String fullDbName = getFullDbName(dbName);
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.badRequest("Database " + dbName + " does not exist");
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         Map<String, Map<String, String>> result = Maps.newHashMap();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java
index c9a87d9..d64a788 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableQueryPlanAction.java
@@ -114,14 +114,10 @@ public class TableQueryPlanAction extends RestBaseController {
             String fullDbName = getFullDbName(dbName);
             // check privilege for select, otherwise return HTTP 401
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tblName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists");
-            }
-            Table table = null;
+            Table table;
             try {
-                // just only support OlapTable, ignore others such as ESTable
-                table = db.getTableOrThrowException(tblName, Table.TableType.OLAP);
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                table = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
             } catch (MetaNotFoundException e) {
                 return ResponseEntityBuilder.okWithCommonError(e.getMessage());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java
index 534e774..8d9d7f9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java
@@ -61,13 +61,10 @@ public class TableRowCountAction extends RestBaseController {
             String fullDbName = getFullDbName(dbName);
             // check privilege for select, otherwise return HTTP 401
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tblName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists");
-            }
-            OlapTable olapTable = null;
+            OlapTable olapTable;
             try {
-                olapTable = (OlapTable) db.getTableOrThrowException(tblName, Table.TableType.OLAP);
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                olapTable = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
             } catch (MetaNotFoundException e) {
                 return ResponseEntityBuilder.okWithCommonError(e.getMessage());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java
index a776fce..856abc1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableSchemaAction.java
@@ -62,13 +62,10 @@ public class TableSchemaAction extends RestBaseController {
             String fullDbName = getFullDbName(dbName);
             // check privilege for select, otherwise return 401 HTTP status
             checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tblName, PrivPredicate.SELECT);
-            Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-            if (db == null) {
-                return ResponseEntityBuilder.okWithCommonError("Database [" + dbName + "] " + "does not exists");
-            }
-            Table table = null;
+            Table table;
             try {
-                table = db.getTableOrThrowException(tblName, Table.TableType.OLAP);
+                Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+                table = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
             } catch (MetaNotFoundException e) {
                 return ResponseEntityBuilder.okWithCommonError(e.getMessage());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/MetaInfoActionV2.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/MetaInfoActionV2.java
index c3f64e2..43e9819 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/MetaInfoActionV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/MetaInfoActionV2.java
@@ -28,6 +28,7 @@ import org.apache.doris.catalog.Table;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
@@ -139,9 +140,11 @@ public class MetaInfoActionV2 extends RestBaseController {
 
 
         String fullDbName = getFullDbName(dbName);
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database does not exist: " + fullDbName);
+        Database db;
+        try {
+            db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
 
         List<String> tblNames = Lists.newArrayList();
@@ -207,30 +210,25 @@ public class MetaInfoActionV2 extends RestBaseController {
 
         String fullDbName = getFullDbName(dbName);
         checkTblAuth(ConnectContext.get().getCurrentUserIdentity(), fullDbName, tblName, PrivPredicate.SHOW);
-
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            return ResponseEntityBuilder.okWithCommonError("Database does not exist: " + fullDbName);
-        }
-
         String withMvPara = request.getParameter(PARAM_WITH_MV);
-        boolean withMv = Strings.isNullOrEmpty(withMvPara) ? false : withMvPara.equals("1");
-
+        boolean withMv = !Strings.isNullOrEmpty(withMvPara) && withMvPara.equals("1");
 
-        TableSchemaInfo tableSchemaInfo = new TableSchemaInfo();
-        db.readLock();
         try {
-            Table tbl = db.getTable(tblName);
-            if (tbl == null) {
-                return ResponseEntityBuilder.okWithCommonError("Table does not exist: " + tblName);
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+            db.readLock();
+            try {
+                Table tbl = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
+
+                TableSchemaInfo tableSchemaInfo = new TableSchemaInfo();
+                tableSchemaInfo.setEngineType(tbl.getType().toString());
+                SchemaInfo schemaInfo = generateSchemaInfo(tbl, withMv);
+                tableSchemaInfo.setSchemaInfo(schemaInfo);
+                return ResponseEntityBuilder.ok(tableSchemaInfo);
+            } finally {
+                db.readUnlock();
             }
-
-            tableSchemaInfo.setEngineType(tbl.getType().toString());
-            SchemaInfo schemaInfo = generateSchemaInfo(tbl, withMv);
-            tableSchemaInfo.setSchemaInfo(schemaInfo);
-            return ResponseEntityBuilder.ok(tableSchemaInfo);
-        } finally {
-            db.readUnlock();
+        } catch (MetaNotFoundException e) {
+            return ResponseEntityBuilder.okWithCommonError(e.getMessage());
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/StatisticAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/StatisticAction.java
index 6fff16f..84436f6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/StatisticAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/restv2/StatisticAction.java
@@ -18,7 +18,6 @@
 package org.apache.doris.httpv2.restv2;
 
 import org.apache.doris.catalog.Catalog;
-import org.apache.doris.catalog.Database;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 import org.apache.doris.httpv2.rest.RestBaseController;
 import org.apache.doris.system.Backend;
@@ -36,6 +35,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 @RestController
 @RequestMapping("/rest/v2")
@@ -60,13 +60,8 @@ public class StatisticAction extends RestBaseController {
     }
 
     private int getTblCount(Catalog catalog) {
-        int tblCount = 0;
-        List<Long> dbIds = catalog.getDbIds();
-        for (long dbId : dbIds) {
-            Database db = catalog.getDb(dbId);
-            tblCount += db.getTables().size();
-        }
-        return tblCount;
+        return catalog.getDbIds().stream().map(catalog::getDbNullable).filter(Objects::nonNull)
+                .map(db -> db.getTables().size()).reduce(Integer::sum).orElse(0);
     }
 
     private long getDiskOccupancy(SystemInfoService infoService) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java
index bc03e6c..b19d312 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/BrokerFileGroup.java
@@ -135,17 +135,9 @@ public class BrokerFileGroup implements Writable {
     // This will parse the input DataDescription to list for BrokerFileInfo
     public void parse(Database db, DataDescription dataDescription) throws DdlException {
         // tableId
-        Table table = db.getTable(dataDescription.getTableName());
-        if (table == null) {
-            throw new DdlException("Unknown table " + dataDescription.getTableName()
-                    + " in database " + db.getFullName());
-        }
-        if (!(table instanceof OlapTable)) {
-            throw new DdlException("Table " + table.getName() + " is not OlapTable");
-        }
-        OlapTable olapTable = (OlapTable) table;
-        tableId = table.getId();
-        table.readLock();
+        OlapTable olapTable = db.getOlapTableOrDdlException(dataDescription.getTableName());
+        tableId = olapTable.getId();
+        olapTable.readLock();
         try {
             // partitionId
             PartitionNames partitionNames = dataDescription.getPartitionNames();
@@ -154,14 +146,14 @@ public class BrokerFileGroup implements Writable {
                 for (String pName : partitionNames.getPartitionNames()) {
                     Partition partition = olapTable.getPartition(pName, partitionNames.isTemp());
                     if (partition == null) {
-                        throw new DdlException("Unknown partition '" + pName + "' in table '" + table.getName() + "'");
+                        throw new DdlException("Unknown partition '" + pName + "' in table '" + olapTable.getName() + "'");
                     }
                     partitionIds.add(partition.getId());
                 }
             }
 
             if (olapTable.getState() == OlapTableState.RESTORE) {
-                throw new DdlException("Table [" + table.getName() + "] is under restore");
+                throw new DdlException("Table [" + olapTable.getName() + "] is under restore");
             }
 
             if (olapTable.getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) {
@@ -170,14 +162,14 @@ public class BrokerFileGroup implements Writable {
 
             // check negative for sum aggregate type
             if (dataDescription.isNegative()) {
-                for (Column column : table.getBaseSchema()) {
+                for (Column column : olapTable.getBaseSchema()) {
                     if (!column.isKey() && column.getAggregationType() != AggregateType.SUM) {
                         throw new DdlException("Column is not SUM AggregateType. column:" + column.getName());
                     }
                 }
             }
         } finally {
-            table.readUnlock();
+            olapTable.readUnlock();
         }
 
         // column
@@ -208,10 +200,7 @@ public class BrokerFileGroup implements Writable {
         if (dataDescription.isLoadFromTable()) {
             String srcTableName = dataDescription.getSrcTableName();
             // src table should be hive table
-            Table srcTable = db.getTable(srcTableName);
-            if (srcTable == null) {
-                throw new DdlException("Unknown table " + srcTableName + " in database " + db.getFullName());
-            }
+            Table srcTable = db.getTableOrDdlException(srcTableName);
             if (!(srcTable instanceof HiveTable)) {
                 throw new DdlException("Source table " + srcTableName + " is not HiveTable");
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
index 36cd639..d23b604 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
@@ -147,26 +147,15 @@ public class DeleteHandler implements Writable {
         List<String> partitionNames = stmt.getPartitionNames();
         boolean noPartitionSpecified = partitionNames.isEmpty();
         List<Predicate> conditions = stmt.getDeleteConditions();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         DeleteJob deleteJob = null;
         try {
             MarkedCountDownLatch<Long, Long> countDownLatch;
             long transactionId = -1;
-            Table table = null;
-            try {
-                table = db.getTableOrThrowException(tableName, Table.TableType.OLAP);
-            } catch (MetaNotFoundException e) {
-                throw new DdlException(e.getMessage());
-            }
-
-            table.readLock();
+            OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
+            olapTable.readLock();
             try {
-                OlapTable olapTable = (OlapTable) table;
-
                 if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) {
                     // table under alter operation can also do delete.
                     // just add a comment here to notice.
@@ -197,7 +186,7 @@ public class DeleteHandler implements Writable {
                         throw new DdlException("Partition does not exist. name: " + partName);
                     }
                     partitions.add(partition);
-                    partitionReplicaNum.put(partition.getId(), ((OlapTable) table).getPartitionInfo().getReplicationNum(partition.getId()));
+                    partitionReplicaNum.put(partition.getId(), olapTable.getPartitionInfo().getReplicationNum(partition.getId()));
                 }
 
                 List<String> deleteConditions = Lists.newArrayList();
@@ -211,7 +200,7 @@ public class DeleteHandler implements Writable {
                 long jobId = Catalog.getCurrentCatalog().getNextId();
                 // begin txn here and generate txn id
                 transactionId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction(db.getId(),
-                        Lists.newArrayList(table.getId()), label, null,
+                        Lists.newArrayList(olapTable.getId()), label, null,
                         new TxnCoordinator(TxnSourceType.FE, FrontendOptions.getLocalHostAddress()),
                         TransactionState.LoadJobSourceType.FRONTEND, jobId, Config.stream_load_default_timeout_second);
 
@@ -289,7 +278,7 @@ public class DeleteHandler implements Writable {
                 }
                 throw new DdlException(t.getMessage(), t);
             } finally {
-                table.readUnlock();
+                olapTable.readUnlock();
             }
 
             long timeoutMs = deleteJob.getTimeoutMs();
@@ -344,14 +333,14 @@ public class DeleteHandler implements Writable {
                             cancelJob(deleteJob, CancelType.UNKNOWN, e.getMessage());
                             throw new DdlException(e.getMessage(), e);
                         }
-                        commitJob(deleteJob, db, table, timeoutMs);
+                        commitJob(deleteJob, db, olapTable, timeoutMs);
                         break;
                     default:
                         Preconditions.checkState(false, "wrong delete job state: " + state.name());
                         break;
                 }
             } else {
-                commitJob(deleteJob, db, table, timeoutMs);
+                commitJob(deleteJob, db, olapTable, timeoutMs);
             }
         } finally {
             if (!FeConstants.runningUnitTest) {
@@ -676,7 +665,7 @@ public class DeleteHandler implements Writable {
     // show delete stmt
     public List<List<Comparable>> getDeleteInfosByDb(long dbId) {
         LinkedList<List<Comparable>> infos = new LinkedList<List<Comparable>>();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             return infos;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java
index c94980b..61dcad5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteJob.java
@@ -18,7 +18,6 @@
 package org.apache.doris.load;
 
 import org.apache.doris.catalog.Catalog;
-import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.FeConstants;
@@ -86,11 +85,7 @@ public class DeleteJob extends AbstractTxnStateChangeCallback {
      */
     public void checkAndUpdateQuorum() throws MetaNotFoundException {
         long dbId = deleteInfo.getDbId();
-        long tableId = deleteInfo.getTableId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("can not find database "+ dbId +" when commit delete");
-        }
+        Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
 
         for (TabletDeleteInfo tDeleteInfo : getTabletDeleteInfo()) {
             Short replicaNum = partitionReplicaNum.get(tDeleteInfo.getPartitionId());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
index 30c0f45..fde11b9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java
@@ -197,11 +197,7 @@ public class ExportJob implements Writable {
 
     public void setJob(ExportStmt stmt) throws UserException {
         String dbName = stmt.getTblName().getDb();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         Preconditions.checkNotNull(stmt.getBrokerDesc());
         this.brokerDesc = stmt.getBrokerDesc();
 
@@ -216,7 +212,7 @@ public class ExportJob implements Writable {
 
         this.partitions = stmt.getPartitions();
 
-        this.exportTable = db.getTable(stmt.getTblName().getTbl());
+        this.exportTable = db.getTableOrDdlException(stmt.getTblName().getTbl());
         this.columns = stmt.getColumns();
         if (!Strings.isNullOrEmpty(this.columns)) {
             Splitter split = Splitter.on(',').trimResults().omitEmptyStrings();
@@ -225,9 +221,6 @@ public class ExportJob implements Writable {
         exportTable.readLock();
         try {
             this.dbId = db.getId();
-            if (exportTable == null) {
-                throw new DdlException("Table " + stmt.getTblName().getTbl() + " does not exist");
-            }
             this.tableId = exportTable.getId();
             this.tableName = stmt.getTblName();
             genExecFragment();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java
index 51019ca..1338002 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportMgr.java
@@ -148,7 +148,7 @@ public class ExportMgr {
                 TableName tableName = job.getTableName();
                 if (tableName == null || tableName.getTbl().equals("DUMMY")) {
                     // forward compatibility, no table name is saved before
-                    Database db = Catalog.getCurrentCatalog().getDb(dbId);
+                    Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
                     if (db == null) {
                         continue;
                     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
index bc78e44..f62d4c4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
@@ -368,10 +368,7 @@ public class Load {
     public void addLoadJob(LoadStmt stmt, EtlJobType etlJobType, long timestamp) throws DdlException {
         // get db
         String dbName = stmt.getLabel().getDbName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         // create job
         LoadJob job = createLoadJob(stmt, etlJobType, db, timestamp);
@@ -392,7 +389,7 @@ public class Load {
         readLock();
         try {
             for (Long tblId : job.getIdToTableLoadInfo().keySet()) {
-                Table tbl = db.getTable(tblId);
+                Table tbl = db.getTableNullable(tblId);
                 if (tbl != null && tbl.getType() == TableType.OLAP
                         && ((OlapTable) tbl).getState() == OlapTableState.RESTORE) {
                     throw new DdlException("Table " + tbl.getName() + " is in restore process. "
@@ -511,10 +508,7 @@ public class Load {
 
             for (DataDescription dataDescription : dataDescriptions) {
                 String tableName = dataDescription.getTableName();
-                OlapTable table = (OlapTable) db.getTable(tableName);
-                if (table == null) {
-                    throw new DdlException("Table[" + tableName + "] does not exist");
-                }
+                OlapTable table = db.getOlapTableOrDdlException(tableName);
 
                 table.readLock();
                 try {
@@ -619,33 +613,27 @@ public class Load {
         String tableName = dataDescription.getTableName();
         Map<String, Pair<String, List<String>>> columnToFunction = null;
 
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            throw new DdlException("Table [" + tableName + "] does not exist");
-        }
+        OlapTable table = db.getOlapTableOrDdlException(tableName);
         tableId = table.getId();
-        if (table.getType() != TableType.OLAP) {
-            throw new DdlException("Table [" + tableName + "] is not olap table");
-        }
 
         table.readLock();
         try {
-            if (((OlapTable) table).getPartitionInfo().isMultiColumnPartition() && jobType == EtlJobType.HADOOP) {
+            if (table.getPartitionInfo().isMultiColumnPartition() && jobType == EtlJobType.HADOOP) {
                 throw new DdlException("Load by hadoop cluster does not support table with multi partition columns."
                         + " Table: " + table.getName() + ". Try using broker load. See 'help broker load;'");
             }
 
             // check partition
             if (dataDescription.getPartitionNames() != null &&
-                    ((OlapTable) table).getPartitionInfo().getType() == PartitionType.UNPARTITIONED) {
+                    table.getPartitionInfo().getType() == PartitionType.UNPARTITIONED) {
                 ErrorReport.reportDdlException(ErrorCode.ERR_PARTITION_CLAUSE_NO_ALLOWED);
             }
 
-            if (((OlapTable) table).getState() == OlapTableState.RESTORE) {
+            if (table.getState() == OlapTableState.RESTORE) {
                 throw new DdlException("Table [" + tableName + "] is under restore");
             }
 
-            if (((OlapTable) table).getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) {
+            if (table.getKeysType() != KeysType.AGG_KEYS && dataDescription.isNegative()) {
                 throw new DdlException("Load for AGG_KEYS table should not specify NEGATIVE");
             }
 
@@ -804,7 +792,7 @@ public class Load {
             }
 
             // partitions of this source
-            OlapTable olapTable = (OlapTable) table;
+            OlapTable olapTable = table;
             PartitionNames partitionNames = dataDescription.getPartitionNames();
             if (partitionNames == null) {
                 for (Partition partition : olapTable.getPartitions()) {
@@ -1450,10 +1438,7 @@ public class Load {
     // return true if we truly register a mini load label
     // return false otherwise (eg: a retry request)
     public boolean registerMiniLabel(String fullDbName, String label, long timestamp) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + fullDbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(fullDbName);
 
         long dbId = db.getId();
         writeLock();
@@ -1480,10 +1465,7 @@ public class Load {
     }
 
     public void deregisterMiniLabel(String fullDbName, String label) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + fullDbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(fullDbName);
 
         long dbId = db.getId();
         writeLock();
@@ -1595,10 +1577,7 @@ public class Load {
 
     public boolean isLabelExist(String dbName, String labelValue, boolean isAccurateMatch) throws DdlException, AnalysisException {
         // get load job and check state
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         readLock();
         try {
             Map<String, List<LoadJob>> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -1636,10 +1615,7 @@ public class Load {
         String label = stmt.getLabel();
 
         // get load job and check state
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         // List of load jobs waiting to be cancelled
         List<LoadJob> loadJobs = Lists.newArrayList();
         readLock();
@@ -1724,11 +1700,8 @@ public class Load {
         String label = stmt.getLabel();
 
         // get load job and check state
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
-        LoadJob job = null;
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
+        LoadJob job;
         readLock();
         try {
             Map<String, List<LoadJob>> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -2112,7 +2085,7 @@ public class Load {
         }
 
         long dbId = loadJob.getDbId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             return infos;
         }
@@ -2130,7 +2103,7 @@ public class Load {
 
                 long tableId = tabletMeta.getTableId();
 
-                OlapTable table = (OlapTable) db.getTable(tableId);
+                OlapTable table = (OlapTable) db.getTableNullable(tableId);
                 if (table == null) {
                     continue;
                 }
@@ -2287,10 +2260,7 @@ public class Load {
     public void getJobInfo(JobInfo info) throws DdlException, MetaNotFoundException {
         String fullDbName = ClusterNamespace.getFullName(info.clusterName, info.dbName);
         info.dbName = fullDbName;
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        if (db == null) {
-            throw new MetaNotFoundException("Unknown database(" + info.dbName + ")");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
         readLock();
         try {
             Map<String, List<LoadJob>> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -2330,7 +2300,7 @@ public class Load {
             Map<Long, ReplicaPersistInfo> replicaInfos = job.getReplicaPersistInfos();
             if (replicaInfos != null) {
                 for (ReplicaPersistInfo info : replicaInfos.values()) {
-                    OlapTable table = (OlapTable) db.getTable(info.getTableId());
+                    OlapTable table = (OlapTable) db.getTableNullable(info.getTableId());
                     if (table == null) {
                         LOG.warn("the table[{}] is missing", info.getIndexId());
                         continue;
@@ -2366,7 +2336,10 @@ public class Load {
             if (idToTableLoadInfo != null) {
                 for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
                     long tableId = tableEntry.getKey();
-                    OlapTable table = (OlapTable) db.getTable(tableId);
+                    OlapTable table = (OlapTable) db.getTableNullable(tableId);
+                    if (table == null) {
+                        continue;
+                    }
                     TableLoadInfo tableLoadInfo = tableEntry.getValue();
                     for (Entry<Long, PartitionLoadInfo> entry : tableLoadInfo.getIdToPartitionLoadInfo().entrySet()) {
                         long partitionId = entry.getKey();
@@ -2403,9 +2376,9 @@ public class Load {
         replaceLoadJob(job);
     }
 
-    public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlException {
+    public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws MetaNotFoundException {
         // TODO: need to call this.writeLock()?
-        Database db = catalog.getDb(job.getDbId());
+        Database db = catalog.getDbOrMetaException(job.getDbId());
 
         List<Long> tableIds = Lists.newArrayList();
         long tblId = job.getTableId();
@@ -2415,13 +2388,7 @@ public class Load {
             tableIds.addAll(job.getIdToTableLoadInfo().keySet());
         }
 
-        List<Table> tables = null;
-        try {
-            tables = db.getTablesOnIdOrderOrThrowException(tableIds);
-        } catch (MetaNotFoundException e) {
-            LOG.error("should not happen", e);
-            return;
-        }
+        List<Table> tables = db.getTablesOnIdOrderOrThrowException(tableIds);
 
         MetaLockUtils.writeLockTables(tables);
         try {
@@ -2446,7 +2413,7 @@ public class Load {
             Map<Long, ReplicaPersistInfo> replicaInfos = job.getReplicaPersistInfos();
             if (replicaInfos != null) {
                 for (ReplicaPersistInfo info : replicaInfos.values()) {
-                    OlapTable table = (OlapTable) db.getTable(info.getTableId());
+                    OlapTable table = (OlapTable) db.getTableNullable(info.getTableId());
                     if (table == null) {
                         LOG.warn("the table[{}] is missing", info.getIndexId());
                         continue;
@@ -2488,9 +2455,9 @@ public class Load {
         replaceLoadJob(job);
     }
 
-    public void replayFinishLoadJob(LoadJob job, Catalog catalog) {
+    public void replayFinishLoadJob(LoadJob job, Catalog catalog) throws MetaNotFoundException {
         // TODO: need to call this.writeLock()?
-        Database db = catalog.getDb(job.getDbId());
+        Database db = catalog.getDbOrMetaException(job.getDbId());
         // After finish, the idToTableLoadInfo in load job will be set to null.
         // We lost table info. So we have to use db lock here.
         db.writeLock();
@@ -2506,9 +2473,9 @@ public class Load {
         }
     }
 
-    public void replayClearRollupInfo(ReplicaPersistInfo info, Catalog catalog) {
-        Database db = catalog.getDb(info.getDbId());
-        OlapTable olapTable = (OlapTable) db.getTable(info.getTableId());
+    public void replayClearRollupInfo(ReplicaPersistInfo info, Catalog catalog) throws MetaNotFoundException {
+        Database db = catalog.getDbOrMetaException(info.getDbId());
+        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = olapTable.getPartition(info.getPartitionId());
@@ -2695,7 +2662,7 @@ public class Load {
                     }
 
                     long dbId = job.getDbId();
-                    Database db = Catalog.getCurrentCatalog().getDb(dbId);
+                    Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
                     if (db == null) {
                         LOG.warn("db does not exist. id: {}", dbId);
                         break;
@@ -2724,11 +2691,11 @@ public class Load {
     public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType cancelType, String msg,
                                       List<String> failedMsg) {
         boolean result = true;
-        JobState srcState = null;
+        JobState srcState;
 
         long jobId = job.getId();
         long dbId = job.getDbId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         String errMsg = msg;
         if (db == null) {
             // if db is null, update job to cancelled
@@ -2844,7 +2811,7 @@ public class Load {
         Map<Long, TableLoadInfo> idToTableLoadInfo = job.getIdToTableLoadInfo();
         for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
             long tableId = tableEntry.getKey();
-            OlapTable table = (OlapTable) db.getTable(tableId);
+            OlapTable table = (OlapTable) db.getTableNullable(tableId);
             if (table == null) {
                 LOG.warn("table does not exist, id: {}", tableId);
                 return false;
@@ -2869,7 +2836,10 @@ public class Load {
         // update partition version and index row count
         for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
             long tableId = tableEntry.getKey();
-            OlapTable table = (OlapTable) db.getTable(tableId);
+            OlapTable table = (OlapTable) db.getTableNullable(tableId);
+            if (table == null) {
+                continue;
+            }
 
             TableLoadInfo tableLoadInfo = tableEntry.getValue();
             for (Entry<Long, PartitionLoadInfo> entry : tableLoadInfo.getIdToPartitionLoadInfo().entrySet()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
index 0b8d847..bf804e6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
@@ -226,7 +226,7 @@ public class LoadChecker extends MasterDaemon {
         Load load = Catalog.getCurrentCatalog().getLoadInstance();
         // get db
         long dbId = job.getDbId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "db does not exist. id: " + dbId);
             return;
@@ -356,14 +356,14 @@ public class LoadChecker extends MasterDaemon {
 
     private Set<Long> submitPushTasks(LoadJob job, Database db) {
         Map<Long, TabletLoadInfo> tabletLoadInfos = job.getIdToTabletLoadInfo();
-        boolean needDecompress = (job.getEtlJobType() == EtlJobType.HADOOP) ? true : false;
+        boolean needDecompress = job.getEtlJobType() == EtlJobType.HADOOP;
         AgentBatchTask batchTask = new AgentBatchTask();
         Set<Long> jobTotalTablets = new HashSet<Long>();
 
         Map<Long, TableLoadInfo> idToTableLoadInfo = job.getIdToTableLoadInfo();
         for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
             long tableId = tableEntry.getKey();
-            OlapTable table = (OlapTable) db.getTable(tableId);
+            OlapTable table = (OlapTable) db.getTableNullable(tableId);
             if (table == null) {
                 LOG.warn("table does not exist. id: {}", tableId);
                 // if table is dropped during load, the the job is failed
@@ -539,7 +539,7 @@ public class LoadChecker extends MasterDaemon {
         // if db is null, cancel load job
         Load load = Catalog.getCurrentCatalog().getLoadInstance();
         long dbId = job.getDbId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             load.cancelLoadJob(job, CancelType.LOAD_RUN_FAIL, "db does not exist. id: " + dbId);
             return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java
index fb713d6..a88c1c2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/StreamLoadRecordMgr.java
@@ -272,7 +272,7 @@ public class StreamLoadRecordMgr extends MasterDaemon {
                     }
 
                     String fullDbName = ClusterNamespace.getFullName(cluster, streamLoadItem.getDb());
-                    Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
+                    Database db = Catalog.getCurrentCatalog().getDbNullable(fullDbName);
                     if (db == null) {
                         String dbName = fullDbName;
                         if (Strings.isNullOrEmpty(streamLoadItem.getCluster())) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java
index 93f60ac..afd153d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BrokerLoadJob.java
@@ -194,7 +194,7 @@ public class BrokerLoadJob extends BulkLoadJob {
                 FileGroupAggKey aggKey = entry.getKey();
                 List<BrokerFileGroup> brokerFileGroups = entry.getValue();
                 long tableId = aggKey.getTableId();
-                OlapTable table = (OlapTable) db.getTable(tableId);
+                OlapTable table = (OlapTable) db.getTableNullable(tableId);
                 // Generate loading task and init the plan of task
                 LoadLoadingTask task = new LoadLoadingTask(db, table, brokerDesc,
                         brokerFileGroups, getDeadlineMs(), getExecMemLimit(),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java
index 77d7e6f..ba80021 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/BulkLoadJob.java
@@ -63,8 +63,10 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.stream.Collectors;
 
 /**
  * parent class of BrokerLoadJob and SparkLoadJob from load stmt
@@ -110,13 +112,10 @@ public abstract class BulkLoadJob extends LoadJob {
     public static BulkLoadJob fromLoadStmt(LoadStmt stmt) throws DdlException {
         // get db id
         String dbName = stmt.getLabel().getDbName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         // create job
-        BulkLoadJob bulkLoadJob = null;
+        BulkLoadJob bulkLoadJob;
         try {
             switch (stmt.getEtlJobType()) {
                 case BROKER:
@@ -165,50 +164,27 @@ public abstract class BulkLoadJob extends LoadJob {
     }
 
     private AuthorizationInfo gatherAuthInfo() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + " has been deleted");
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
         return new AuthorizationInfo(database.getFullName(), getTableNames());
     }
 
     @Override
     public Set<String> getTableNamesForShow() {
-        Set<String> result = Sets.newHashSet();
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            for (long tableId : fileGroupAggInfo.getAllTableIds()) {
-                result.add(String.valueOf(tableId));
-            }
-            return result;
-        }
-        for (long tableId : fileGroupAggInfo.getAllTableIds()) {
-            Table table = database.getTable(tableId);
-            if (table == null) {
-                result.add(String.valueOf(tableId));
-            } else {
-                result.add(table.getName());
-            }
-        }
-        return result;
+        Optional<Database> db = Catalog.getCurrentCatalog().getDb(dbId);
+        return fileGroupAggInfo.getAllTableIds().stream()
+                .map(tableId -> db.flatMap(d -> d.getTable(tableId)).map(Table::getName).orElse(String.valueOf(tableId)))
+                .collect(Collectors.toSet());
     }
 
     @Override
     public Set<String> getTableNames() throws MetaNotFoundException {
         Set<String> result = Sets.newHashSet();
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
         // The database will not be locked in here.
         // The getTable is a thread-safe method called without read lock of database
         for (long tableId : fileGroupAggInfo.getAllTableIds()) {
-            Table table = database.getTable(tableId);
-            if (table == null) {
-                throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId);
-            } else {
-                result.add(table.getName());
-            }
+            Table table = database.getTableOrMetaException(tableId);
+            result.add(table.getName());
         }
         return result;
     }
@@ -274,12 +250,9 @@ public abstract class BulkLoadJob extends LoadJob {
         fileGroupAggInfo = new BrokerFileGroupAggInfo();
         SqlParser parser = new SqlParser(new SqlScanner(new StringReader(originStmt.originStmt),
                 Long.valueOf(sessionVariables.get(SessionVariable.SQL_MODE))));
-        LoadStmt stmt = null;
+        LoadStmt stmt;
         try {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
-            if (db == null) {
-                throw new DdlException("Database[" + dbId + "] does not exist");
-            }
+            Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbId);
             stmt = (LoadStmt) SqlParserUtils.getStmt(parser, originStmt.idx);
             for (DataDescription dataDescription : stmt.getDataDescriptions()) {
                 dataDescription.analyzeWithoutCheckPriv(db.getFullName());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java
index bc0db60..8c2bdce 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/InsertLoadJob.java
@@ -68,38 +68,21 @@ public class InsertLoadJob extends LoadJob {
     }
 
     public AuthorizationInfo gatherAuthInfo() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
         return new AuthorizationInfo(database.getFullName(), getTableNames());
     }
 
     @Override
     public Set<String> getTableNamesForShow() {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            return Sets.newHashSet(String.valueOf(tableId));
-        }
-        // The database will not be locked in here.
-        // The getTable is a thread-safe method called without read lock of database
-        Table table = database.getTable(tableId);
-        if (table == null) {
-            return Sets.newHashSet(String.valueOf(tableId));
-        }
-        return Sets.newHashSet(table.getName());
+        String name = Catalog.getCurrentCatalog().getDb(dbId).flatMap(db -> db.getTable(tableId))
+                .map(Table::getName).orElse(String.valueOf(tableId));
+        return Sets.newHashSet(name);
     }
 
     @Override
     public Set<String> getTableNames() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
-        Table table = database.getTable(tableId);
-        if (table == null) {
-            throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId);
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        Table table = database.getTableOrMetaException(tableId);
         return Sets.newHashSet(table.getName());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java
index e33e98b..c5c97b3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadJob.java
@@ -241,12 +241,7 @@ public abstract class LoadJob extends AbstractTxnStateChangeCallback implements
     }
 
     public Database getDb() throws MetaNotFoundException {
-        // get db
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("Database " + dbId + " already has been deleted");
-        }
-        return db;
+        return Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
     }
 
     public long getDbId() {
@@ -565,10 +560,7 @@ public abstract class LoadJob extends AbstractTxnStateChangeCallback implements
      * @throws DdlException
      */
     private void checkAuthWithoutAuthInfo(String command) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbId);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbId);
 
         // check auth
         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java
index b8d8cab..2e7e79a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadManager.java
@@ -148,8 +148,7 @@ public class LoadManager implements Writable{
             cluster = request.getCluster();
         }
         Database database = checkDb(ClusterNamespace.getFullName(cluster, request.getDb()));
-        Table table = database.getTable(request.tbl);
-        checkTable(database, request.getTbl());
+        Table table = database.getTableOrDdlException(request.tbl);
         LoadJob loadJob = null;
         writeLock();
         try {
@@ -279,10 +278,7 @@ public class LoadManager implements Writable{
             long createTimestamp, String failMsg, String trackingUrl) throws MetaNotFoundException {
 
         // get db id
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new MetaNotFoundException("Database[" + dbName + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbName);
 
         LoadJob loadJob;
         switch (jobType) {
@@ -298,10 +294,7 @@ public class LoadManager implements Writable{
     }
 
     public void cancelLoadJob(CancelLoadStmt stmt, boolean isAccurateMatch) throws DdlException, AnalysisException {
-        Database db = Catalog.getCurrentCatalog().getDb(stmt.getDbName());
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + stmt.getDbName());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(stmt.getDbName());
 
         // List of load jobs waiting to be cancelled
         List<LoadJob> loadJobs = Lists.newArrayList();
@@ -355,10 +348,7 @@ public class LoadManager implements Writable{
     }
 
     public void cancelLoadJob(CancelLoadStmt stmt) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(stmt.getDbName());
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + stmt.getDbName());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(stmt.getDbName());
 
         LoadJob loadJob = null;
         readLock();
@@ -615,26 +605,7 @@ public class LoadManager implements Writable{
     }
 
     private Database checkDb(String dbName) throws DdlException {
-        // get db
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            LOG.warn("Database {} does not exist", dbName);
-            throw new DdlException("Database[" + dbName + "] does not exist");
-        }
-        return db;
-    }
-
-    /**
-     * Please don't lock any load lock before check table
-     * @param database
-     * @param tableName
-     * @throws DdlException
-     */
-    private void checkTable(Database database, String tableName) throws DdlException {
-        if (database.getTable(tableName) == null) {
-            LOG.info("Table {} is not belongs to database {}", tableName, database.getFullName());
-            throw new DdlException("Table[" + tableName + "] does not exist");
-        }
+        return Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
     }
 
     /**
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java
index b490dca..a714475 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/LoadingTaskPlanner.java
@@ -97,7 +97,7 @@ public class LoadingTaskPlanner {
         this.sendBatchParallelism = sendBatchParallelism;
         this.userInfo = userInfo;
         if (Catalog.getCurrentCatalog().getAuth().checkDbPriv(userInfo,
-                Catalog.getCurrentCatalog().getDb(dbId).getFullName(), PrivPredicate.SELECT)) {
+                Catalog.getCurrentCatalog().getDbNullable(dbId).getFullName(), PrivPredicate.SELECT)) {
             this.analyzer.setUDFAllowed(true);
         } else {
             this.analyzer.setUDFAllowed(false);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/MiniLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/MiniLoadJob.java
index 705e3f6..65a9a85 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/MiniLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/MiniLoadJob.java
@@ -83,10 +83,7 @@ public class MiniLoadJob extends LoadJob {
     }
 
     public AuthorizationInfo gatherAuthInfo() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
         return new AuthorizationInfo(database.getFullName(), getTableNames());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
index 85e1eae..41d850b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
@@ -131,16 +131,13 @@ public class SparkLoadPendingTask extends LoadTask {
     }
 
     private void createEtlJobConf() throws LoadException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new LoadException("db does not exist. id: " + dbId);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbId, s -> new LoadException("db does not exist. id: " + s));
 
         Map<Long, EtlTable> tables = Maps.newHashMap();
         Map<Long, Set<Long>> tableIdToPartitionIds = Maps.newHashMap();
         Set<Long> allPartitionsTableIds = Sets.newHashSet();
         prepareTablePartitionInfos(db, tableIdToPartitionIds, allPartitionsTableIds);
-        List<Table> tableList = null;
+        List<Table> tableList;
         try {
             tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(allPartitionsTableIds));
         } catch (MetaNotFoundException e) {
@@ -153,10 +150,7 @@ public class SparkLoadPendingTask extends LoadTask {
                 FileGroupAggKey aggKey = entry.getKey();
                 long tableId = aggKey.getTableId();
 
-                OlapTable table = (OlapTable) db.getTable(tableId);
-                if (table == null) {
-                    throw new LoadException("table does not exist. id: " + tableId);
-                }
+                OlapTable table = (OlapTable) db.getTableOrException(tableId, s -> new LoadException("table does not exist. id: " + s));
 
                 EtlTable etlTable = null;
                 if (tables.containsKey(tableId)) {
@@ -204,13 +198,10 @@ public class SparkLoadPendingTask extends LoadTask {
                 continue;
             }
 
-            OlapTable table = (OlapTable) db.getTable(tableId);
-            if (table == null) {
-                throw new LoadException("table does not exist. id: " + tableId);
-            }
+            OlapTable table = (OlapTable) db.getTableOrException(tableId, s -> new LoadException("table does not exist. id: " + s));
             table.readLock();
             try {
-                Set<Long> partitionIds = null;
+                Set<Long> partitionIds;
                 if (tableIdToPartitionIds.containsKey(tableId)) {
                     partitionIds = tableIdToPartitionIds.get(tableId);
                 } else {
@@ -486,10 +477,7 @@ public class SparkLoadPendingTask extends LoadTask {
         Map<String, String> hiveTableProperties = Maps.newHashMap();
         if (fileGroup.isLoadFromTable()) {
             long srcTableId = fileGroup.getSrcTableId();
-            HiveTable srcHiveTable = (HiveTable) db.getTable(srcTableId);
-            if (srcHiveTable == null) {
-                throw new LoadException("table does not exist. id: " + srcTableId);
-            }
+            HiveTable srcHiveTable = (HiveTable) db.getTableOrException(srcTableId, s -> new LoadException("table does not exist. id: " + s));
             hiveDbTableName = srcHiveTable.getHiveDbTable();
             hiveTableProperties.putAll(srcHiveTable.getHiveProperties());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java
index a357055..7a3b87b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java
@@ -22,11 +22,9 @@ import org.apache.doris.analysis.CreateRoutineLoadStmt;
 import org.apache.doris.analysis.RoutineLoadDataSourceProperties;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
-import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.InternalErrorCode;
 import org.apache.doris.common.LoadException;
@@ -356,14 +354,10 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob {
 
     public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws UserException {
         // check db and table
-        Database db = Catalog.getCurrentCatalog().getDb(stmt.getDBName());
-        if (db == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDBName());
-        }
-
-        checkMeta(db, stmt.getTableName(), stmt.getRoutineLoadDesc());
-        Table table = db.getTable(stmt.getTableName());
-        long tableId = table.getId();
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(stmt.getDBName());
+        OlapTable olapTable = db.getOlapTableOrDdlException(stmt.getTableName());
+        checkMeta(olapTable, stmt.getRoutineLoadDesc());
+        long tableId = olapTable.getId();
 
         // init kafka routine load job
         long id = Catalog.getCurrentCatalog().getNextId();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java
index 07854b7..f4d4d6e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java
@@ -20,7 +20,6 @@ package org.apache.doris.load.routineload;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.Table;
-import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.DebugUtil;
 import org.apache.doris.thrift.TExecPlanFragmentParams;
@@ -71,15 +70,9 @@ public class KafkaTaskInfo extends RoutineLoadTaskInfo {
         tRoutineLoadTask.setId(queryId);
         tRoutineLoadTask.setJobId(jobId);
         tRoutineLoadTask.setTxnId(txnId);
-        Database database = Catalog.getCurrentCatalog().getDb(routineLoadJob.getDbId());
-        if (database == null) {
-            throw new MetaNotFoundException("database " + routineLoadJob.getDbId() + " does not exist");
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(routineLoadJob.getDbId());
+        Table tbl = database.getTableOrMetaException(routineLoadJob.getTableId());
         tRoutineLoadTask.setDb(database.getFullName());
-        Table tbl = database.getTable(routineLoadJob.getTableId());
-        if (tbl == null) {
-            throw new MetaNotFoundException("table " + routineLoadJob.getTableId() + " does not exist");
-        }
         tRoutineLoadTask.setTbl(tbl.getName());
         // label = job_name+job_id+task_id+txn_id
         String label = Joiner.on("-").join(routineLoadJob.getName(), routineLoadJob.getId(), DebugUtil.printId(id), txnId);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
index edca511..734c8b3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java
@@ -31,11 +31,8 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Table;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
-import org.apache.doris.common.ErrorCode;
-import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.InternalErrorCode;
 import org.apache.doris.common.MetaNotFoundException;
@@ -404,12 +401,7 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     }
 
     public String getDbFullName() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
-
-        return database.getFullName();
+        return Catalog.getCurrentCatalog().getDbOrMetaException(dbId).getFullName();
     }
 
     public long getTableId() {
@@ -417,17 +409,8 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     }
 
     public String getTableName() throws MetaNotFoundException {
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
-        if (database == null) {
-            throw new MetaNotFoundException("Database " + dbId + "has been deleted");
-        }
-
-        Table table = database.getTable(tableId);
-        if (table == null) {
-            throw new MetaNotFoundException("Failed to find table " + tableId + " in db " + dbId);
-        }
-        return table.getName();
-
+        Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        return database.getTableOrMetaException(tableId).getName();
     }
 
     public JobState getState() {
@@ -789,20 +772,14 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     }
 
     private void initPlanner() throws UserException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("db " + dbId + " does not exist");
-        }
-        planner = new StreamLoadPlanner(db, (OlapTable) db.getTable(this.tableId), this);
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        planner = new StreamLoadPlanner(db, db.getTableOrMetaException(this.tableId, Table.TableType.OLAP), this);
     }
 
     public TExecPlanFragmentParams plan(TUniqueId loadId, long txnId) throws UserException {
         Preconditions.checkNotNull(planner);
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("db " + dbId + " does not exist");
-        }
-        Table table = db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        Table table = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         table.readLock();
         try {
             TExecPlanFragmentParams planParams = planner.plan(loadId);
@@ -1091,17 +1068,7 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
         }
     }
 
-    protected static void checkMeta(Database db, String tblName, RoutineLoadDesc routineLoadDesc)
-            throws UserException {
-        Table table = db.getTable(tblName);
-        if (table == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName);
-        }
-
-        if (table.getType() != Table.TableType.OLAP) {
-            throw new AnalysisException("Only olap table support routine load");
-        }
-        
+    protected static void checkMeta(OlapTable olapTable, RoutineLoadDesc routineLoadDesc) throws UserException {
         if (routineLoadDesc == null) {
             return;
         }
@@ -1112,7 +1079,6 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
         }
         
         // check partitions
-        OlapTable olapTable = (OlapTable) table;
         olapTable.readLock();
         try {
             for (String partName : partitionNames.getPartitionNames()) {
@@ -1211,7 +1177,7 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
 
     public void update() throws UserException {
         // check if db and table exist
-        Database database = Catalog.getCurrentCatalog().getDb(dbId);
+        Database database = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (database == null) {
             LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id)
                              .add("db_id", dbId)
@@ -1230,7 +1196,7 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
         }
 
         // check table belong to database
-        Table table = database.getTable(tableId);
+        Table table = database.getTableNullable(tableId);
         if (table == null) {
             LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id).add("db_id", dbId)
                              .add("table_id", tableId)
@@ -1281,8 +1247,8 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     protected abstract String getStatistic();
 
     public List<String> getShowInfo() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        Table tbl = (db == null) ? null : db.getTable(tableId);
+        Optional<Database> database = Catalog.getCurrentCatalog().getDb(dbId);
+        Optional<Table> table = database.flatMap(db -> db.getTable(tableId));
 
         readLock();
         try {
@@ -1292,8 +1258,8 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
             row.add(TimeUtils.longToTimeString(createTimestamp));
             row.add(TimeUtils.longToTimeString(pauseTimestamp));
             row.add(TimeUtils.longToTimeString(endTimestamp));
-            row.add(db == null ? String.valueOf(dbId) : db.getFullName());
-            row.add(tbl == null ? String.valueOf(tableId) : tbl.getName());
+            row.add(database.map(Database::getFullName).orElse(String.valueOf(dbId)));
+            row.add(table.map(Table::getName).orElse(String.valueOf(tableId)));
             row.add(getState().name());
             row.add(dataSourceType.name());
             row.add(String.valueOf(getSizeOfRoutineLoadTaskInfoList()));
@@ -1327,13 +1293,13 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     }
 
     public String getShowCreateInfo() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        Table tbl = (db == null) ? null : db.getTable(tableId);
+        Optional<Database> database = Catalog.getCurrentCatalog().getDb(dbId);
+        Optional<Table> table = database.flatMap(db -> db.getTable(tableId));
         StringBuilder sb = new StringBuilder();
         // 1.job_name
         sb.append("CREATE ROUTINE LOAD ").append(name);
         // 2.tbl_name
-        sb.append(" ON ").append(tbl == null ? String.valueOf(tableId) : tbl.getName()).append("\n");
+        sb.append(" ON ").append(table.map(Table::getName).orElse(String.valueOf(tableId))).append("\n");
         // 3.merge_type
         sb.append("WITH ").append(mergeType.name()).append("\n");
         // 4.load_properties
@@ -1420,12 +1386,12 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
     }
 
     public List<String> getShowStatistic() {
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Optional<Database> database = Catalog.getCurrentCatalog().getDb(dbId);
 
         List<String> row = Lists.newArrayList();
         row.add(name);
         row.add(String.valueOf(id));
-        row.add(db == null ? String.valueOf(dbId) : db.getFullName());
+        row.add(database.map(Database::getFullName).orElse(String.valueOf(dbId)));
         row.add(getStatistic());
         row.add(getTaskStatistic());
         return row;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
index 0f0a123..e6a5263 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java
@@ -237,10 +237,7 @@ public class RoutineLoadManager implements Writable {
             throws MetaNotFoundException, DdlException, AnalysisException {
 
         List<RoutineLoadJob> result = Lists.newArrayList();
-        Database database = Catalog.getCurrentCatalog().getDb(dbName);
-        if (database == null) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         long dbId = database.getId();
         Map<String, List<RoutineLoadJob>> jobMap = dbToNameToRoutineLoadJob.get(dbId);
         if (jobMap == null) {
@@ -499,12 +496,8 @@ public class RoutineLoadManager implements Writable {
                 break RESULT;
             }
 
-            long dbId = 0L;
-            Database database = Catalog.getCurrentCatalog().getDb(dbFullName);
-            if (database == null) {
-                throw new MetaNotFoundException("failed to find database by dbFullName " + dbFullName);
-            }
-            dbId = database.getId();
+            Database database = Catalog.getCurrentCatalog().getDbOrMetaException(dbFullName);
+            long dbId = database.getId();
             if (!dbToNameToRoutineLoadJob.containsKey(dbId)) {
                 result = new ArrayList<>();
                 break RESULT;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJob.java
index 1077734..3ce5b48 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJob.java
@@ -91,11 +91,8 @@ public abstract class SyncJob implements Writable {
 
     public static SyncJob fromStmt(long jobId, CreateDataSyncJobStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Database " + dbName + " does not exist");
-        }
-        SyncJob syncJob = null;
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
+        SyncJob syncJob;
         try {
             switch (stmt.getDataSyncJobType()) {
                 case CANAL:
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJobManager.java
index 60a6019..8b06f06 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJobManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/sync/SyncJobManager.java
@@ -88,10 +88,7 @@ public class SyncJobManager implements Writable {
         String dbName = stmt.getDbFullName();
         String jobName = stmt.getJobName();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         List<SyncJob> syncJobs = Lists.newArrayList();
         readLock();
@@ -101,7 +98,7 @@ public class SyncJobManager implements Writable {
                 throw new DdlException("Load job does not exist");
             }
 
-            List<SyncJob> runningSyncJob = matchJobs.stream().filter(entity -> entity.isRunning())
+            List<SyncJob> runningSyncJob = matchJobs.stream().filter(SyncJob::isRunning)
                     .collect(Collectors.toList());
             if (runningSyncJob.isEmpty()) {
                 throw new DdlException("There is no running job with jobName `"
@@ -122,10 +119,7 @@ public class SyncJobManager implements Writable {
         String dbName = stmt.getDbFullName();
         String jobName = stmt.getJobName();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         List<SyncJob> syncJobs = Lists.newArrayList();
         readLock();
@@ -135,7 +129,7 @@ public class SyncJobManager implements Writable {
                 throw new DdlException("Load job does not exist");
             }
 
-            List<SyncJob> pausedSyncJob = matchJobs.stream().filter(entity -> entity.isPaused())
+            List<SyncJob> pausedSyncJob = matchJobs.stream().filter(SyncJob::isPaused)
                     .collect(Collectors.toList());
             if (pausedSyncJob.isEmpty()) {
                 throw new DdlException("There is no paused job with jobName `"
@@ -156,10 +150,7 @@ public class SyncJobManager implements Writable {
         String dbName = stmt.getDbFullName();
         String jobName = stmt.getJobName();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         // List of sync jobs waiting to be cancelled
         List<SyncJob> syncJobs = Lists.newArrayList();
@@ -237,10 +228,7 @@ public class SyncJobManager implements Writable {
     }
 
     public boolean isJobNameExist(String dbName, String jobName) throws DdlException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new DdlException("Db does not exist. name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
         boolean result = false;
         readLock();
         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncJob.java
index 1cb3c77..dce10ba 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/sync/canal/CanalSyncJob.java
@@ -24,7 +24,6 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.OlapTable;
-import org.apache.doris.catalog.Table;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.TimeUtils;
@@ -94,29 +93,23 @@ public class CanalSyncJob extends SyncJob {
         if (channels == null) {
             channels = Lists.newArrayList();
         }
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new DdlException("Database[" + dbId + "] does not exist");
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbId);
         db.writeLock();
         try {
             for (ChannelDescription channelDescription : channelDescriptions) {
                 String tableName = channelDescription.getTargetTable();
-                Table table = db.getTable(tableName);
-                if (!(table instanceof OlapTable)) {
-                    throw new DdlException("Table[" + tableName + "] is invalid.");
-                }
-                if (((OlapTable) table).getKeysType() != KeysType.UNIQUE_KEYS || !((OlapTable) table).hasDeleteSign()) {
+                OlapTable olapTable = db.getOlapTableOrDdlException(tableName);
+                if (olapTable.getKeysType() != KeysType.UNIQUE_KEYS || !olapTable.hasDeleteSign()) {
                     throw new DdlException("Table[" + tableName + "] don't support batch delete.");
                 }
                 List<String> colNames = channelDescription.getColNames();
                 if (colNames == null) {
                     colNames = Lists.newArrayList();
-                    for (Column column : table.getBaseSchema(false)) {
+                    for (Column column : olapTable.getBaseSchema(false)) {
                         colNames.add(column.getName());
                     }
                 }
-                CanalSyncChannel syncChannel = new CanalSyncChannel(this, db, (OlapTable) table, colNames,
+                CanalSyncChannel syncChannel = new CanalSyncChannel(this, db, olapTable, colNames,
                         channelDescription.getSrcDatabase(), channelDescription.getSrcTableName());
                 if (channelDescription.getPartitionNames() != null) {
                     syncChannel.setPartitions(channelDescription.getPartitionNames());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/update/UpdateStmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/load/update/UpdateStmtExecutor.java
index 1e8988d..b31f544 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/update/UpdateStmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/update/UpdateStmtExecutor.java
@@ -195,8 +195,10 @@ public class UpdateStmtExecutor {
         boolean isPublished;
         try {
             LOG.info("commit and publish transaction for update stmt, query id: {}", DebugUtil.printId(queryId));
-            isPublished = globalTransactionMgr.commitAndPublishTransaction(Catalog.getCurrentCatalog().getDb(dbId),
-                    Lists.newArrayList(targetTable), txnId,
+            isPublished = globalTransactionMgr.commitAndPublishTransaction(
+                    Catalog.getCurrentCatalog().getDbOrMetaException(dbId),
+                    Lists.newArrayList(targetTable),
+                    txnId,
                     TabletCommitInfo.fromThrift(coordinator.getCommitInfos()),
                     analyzer.getContext().getSessionVariable().getInsertVisibleTimeoutMs());
         } catch (Throwable e) {
@@ -236,12 +238,7 @@ public class UpdateStmtExecutor {
         updateStmtExecutor.targetTable = (OlapTable) updateStmt.getTargetTable();
         updateStmtExecutor.whereExpr = updateStmt.getWhereExpr();
         updateStmtExecutor.setExprs = updateStmt.getSetExprs();
-        Database database = Catalog.getCurrentCatalog().getDb(updateStmt.getTableName().getDb());
-        if (database == null) {
-            String errMsg = "Database does not exists in update stmt, db:" + updateStmt.getTableName().getDb();
-            LOG.info(errMsg);
-            throw new AnalysisException(errMsg);
-        }
+        Database database = Catalog.getCurrentCatalog().getDbOrAnalysisException(updateStmt.getTableName().getDb());
         updateStmtExecutor.dbId = database.getId();
         updateStmtExecutor.analyzer = updateStmt.getAnalyzer();
         updateStmtExecutor.queryId = updateStmtExecutor.analyzer.getContext().queryId();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java
index 2b3217f..55de348 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/master/MasterImpl.java
@@ -305,7 +305,7 @@ public class MasterImpl {
         long backendId = pushTask.getBackendId();
         long signature = task.getSignature();
         long transactionId = ((PushTask) task).getTransactionId();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature);
             return;
@@ -337,7 +337,7 @@ public class MasterImpl {
         }
         LOG.debug("push report state: {}", pushState.name());
 
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
         if (olapTable == null) {
             AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature);
             LOG.warn("finish push replica error, cannot find table[" + tableId + "] when push finished");
@@ -516,7 +516,7 @@ public class MasterImpl {
         long dbId = pushTask.getDbId();
         long backendId = pushTask.getBackendId();
         long signature = task.getSignature();
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             AgentTaskQueue.removePushTask(backendId, signature, finishVersion, finishVersionHash, 
                                           pushTask.getPushType(), pushTask.getTaskType());
@@ -551,13 +551,12 @@ public class MasterImpl {
 
         LOG.debug("push report state: {}", pushState.name());
 
-        OlapTable olapTable = (OlapTable) db.getTable(tableId);
+        OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
         if (olapTable == null) {
             AgentTaskQueue.removeTask(backendId, TTaskType.REALTIME_PUSH, signature);
             LOG.warn("finish push replica error, cannot find table[" + tableId + "] when push finished");
             return;
         }
-
         olapTable.writeLock();
         try {
             Partition partition = olapTable.getPartition(partitionId);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java
index 2181ff5..8a79f3d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java
@@ -50,7 +50,7 @@ public class PartitionInMemoryInfoCollector extends MasterDaemon {
         ImmutableSet.Builder builder = ImmutableSet.builder();
         List<Long> dbIdList = catalog.getDbIds();
         for (Long dbId : dbIdList) {
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 LOG.warn("Database [" + dbId + "] does not exist, skip to update database used data quota");
                 continue;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
index fe94732..867c634 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
@@ -398,7 +398,7 @@ public class ReportHandler extends Daemon {
                              long backendId, long backendReportVersion) {
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
         for (Long dbId : tabletSyncMap.keySet()) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -414,7 +414,7 @@ public class ReportHandler extends Daemon {
                 }
                 long tabletId = tabletIds.get(i);
                 long tableId = tabletMeta.getTableId();
-                OlapTable olapTable = (OlapTable) db.getTable(tableId);
+                OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
                 if (olapTable == null) {
                     continue;
                 }
@@ -528,7 +528,7 @@ public class ReportHandler extends Daemon {
         AgentBatchTask createReplicaBatchTask = new AgentBatchTask();
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
         for (Long dbId : tabletDeleteFromMeta.keySet()) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -543,7 +543,7 @@ public class ReportHandler extends Daemon {
                 }
                 long tabletId = tabletIds.get(i);
                 long tableId = tabletMeta.getTableId();
-                OlapTable olapTable = (OlapTable) db.getTable(tableId);
+                OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
                 if (olapTable == null) {
                     continue;
                 }
@@ -802,7 +802,7 @@ public class ReportHandler extends Daemon {
         BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId);
         backendTabletsInfo.setBad(true);
         for (Long dbId : tabletRecoveryMap.keySet()) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -815,7 +815,7 @@ public class ReportHandler extends Daemon {
                 }
                 long tabletId = tabletIds.get(i);
                 long tableId = tabletMeta.getTableId();
-                OlapTable olapTable = (OlapTable) db.getTable(tableId);
+                OlapTable olapTable = (OlapTable) db.getTableNullable(tableId);
                 if (olapTable == null) {
                     continue;
                 }
@@ -936,12 +936,8 @@ public class ReportHandler extends Daemon {
         long dataSize = backendTabletInfo.getDataSize();
         long rowCount = backendTabletInfo.getRowCount();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("db[" + dbId + "] does not exist");
-        }
-
-        OlapTable olapTable = (OlapTable) db.getTableOrThrowException(tableId, Table.TableType.OLAP);
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
+        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = olapTable.getPartition(partitionId);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
index c0f562b..cc84395 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
@@ -40,6 +40,7 @@ import org.apache.doris.cluster.BaseParam;
 import org.apache.doris.cluster.Cluster;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.SmallFileMgr.SmallFile;
@@ -194,11 +195,7 @@ public class EditLog {
                 }
                 case OperationType.OP_DROP_TABLE: {
                     DropInfo info = (DropInfo) journal.getData();
-                    Database db = catalog.getDb(info.getDbId());
-                    if (db == null) {
-                        LOG.warn("failed to get db[{}]", info.getDbId());
-                        break;
-                    }
+                    Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getDbId());
                     LOG.info("Begin to unprotect drop table. db = "
                             + db.getFullName() + " table = " + info.getTableId());
                     catalog.replayDropTable(db, info.getTableId(), info.isForceDrop());
@@ -865,6 +862,20 @@ public class EditLog {
                     throw e;
                 }
             }
+        } catch (MetaNotFoundException e) {
+            /**
+             * In the following cases, doris may record metadata modification information for a table that no longer exists.
+             * 1. Thread 1: get TableA object
+             * 2. Thread 2: lock db and drop table and record edit log of the dropped TableA
+             * 3. Thread 1: lock table, modify table and record edit log of the modified TableA
+             * **The modified edit log is after the dropped edit log**
+             * Because the table has been dropped, the olapTable in here is null when the modified edit log is replayed.
+             * So in this case, we will ignore the edit log of the modified table after the table is dropped.
+             * This could make the meta inconsistent, for example, an edit log on a dropped table is ignored, but
+             * this table is restored later, so there may be an inconsistent situation between master and followers. We
+             * log a warning here to debug when happens. This could happen to other meta like DB.
+             */
+            LOG.warn("[INCONSISTENT META] replay failed {}: {}", journal, e.getMessage(), e);
         } catch (Exception e) {
             LOG.error("Operation Type {}", opCode, e);
             System.exit(-1);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
index 6a00395..36b4641 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
@@ -369,12 +369,7 @@ public class ConnectContext {
 
     public void setDatabase(String db) {
         currentDb = db;
-        Database database = Catalog.getCurrentCatalog().getDb(db);
-        if (database == null) {
-            currentDbId = -1;
-        } else {
-            currentDbId = database.getId();
-        }
+        currentDbId = Catalog.getCurrentCatalog().getDb(db).map(Database::getId).orElse(-1L);
     }
 
     public void setExecutor(StmtExecutor executor) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
index 056b890..a788efc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
@@ -301,12 +301,12 @@ public class ConnectProcessor {
             ctx.getState().setError("Empty tableName");
             return;
         }
-        Database db = ctx.getCatalog().getDb(ctx.getDatabase());
+        Database db = ctx.getCatalog().getDbNullable(ctx.getDatabase());
         if (db == null) {
             ctx.getState().setError("Unknown database(" + ctx.getDatabase() + ")");
             return;
         }
-        Table table = db.getTable(tableName);
+        Table table = db.getTableNullable(tableName);
         if (table == null) {
             ctx.getState().setError("Unknown table(" + tableName + ")");
             return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index bc3ba5e..06d4859 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -379,10 +379,7 @@ public class ShowExecutor {
     // Handle show functions
     private void handleShowFunctions() throws AnalysisException {
         ShowFunctionsStmt showStmt = (ShowFunctionsStmt) stmt;
-        Database db = ctx.getCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDbName());
         List<Function> functions = showStmt.getIsBuiltin() ? ctx.getCatalog().getBuiltinFunctions() :
             db.getFunctions();
 
@@ -426,10 +423,7 @@ public class ShowExecutor {
     // Handle show create function
     private void handleShowCreateFunction() throws AnalysisException {
         ShowCreateFunctionStmt showCreateFunctionStmt = (ShowCreateFunctionStmt) stmt;
-        Database db = ctx.getCatalog().getDb(showCreateFunctionStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showCreateFunctionStmt.getDbName());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showCreateFunctionStmt.getDbName());
 
         Function function = db.getFunction(showCreateFunctionStmt.getFunction());
         List<List<String>> resultRowSet = Lists.newArrayList();
@@ -443,10 +437,7 @@ public class ShowExecutor {
     // Handle show encryptkeys
     private void handleShowEncryptKeys() throws AnalysisException {
         ShowEncryptKeysStmt showStmt = (ShowEncryptKeysStmt) stmt;
-        Database db = ctx.getCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDbName());
         List<EncryptKey> encryptKeys = db.getEncryptKeys();
 
         List<List<Comparable>> rowSet = Lists.newArrayList();
@@ -541,7 +532,7 @@ public class ShowExecutor {
         long dbId = showStmt.getDbId();
         List<List<String>> rows = Lists.newArrayList();
         Catalog catalog = ctx.getCatalog();
-        Database database = catalog.getDb(dbId);
+        Database database = catalog.getDbNullable(dbId);
         if (database != null) {
             List<String> row = new ArrayList<>();
             row.add(database.getFullName());
@@ -556,12 +547,13 @@ public class ShowExecutor {
         List<List<String>> rows = Lists.newArrayList();
         Catalog catalog = ctx.getCatalog();
         List<Long> dbIds = catalog.getDbIds();
+        // TODO should use inverted index
         for (long dbId : dbIds) {
-            Database database = catalog.getDb(dbId);
+            Database database = catalog.getDbNullable(dbId);
             if (database == null) {
                 continue;
             }
-            Table table = database.getTable(tableId);
+            Table table = database.getTableNullable(tableId);
             if (table != null) {
                 List<String> row = new ArrayList<>();
                 row.add(database.getFullName());
@@ -580,8 +572,9 @@ public class ShowExecutor {
         List<List<String>> rows = Lists.newArrayList();
         Catalog catalog = ctx.getCatalog();
         List<Long> dbIds = catalog.getDbIds();
+        // TODO should use inverted index
         for (long dbId : dbIds) {
-            Database database = catalog.getDb(dbId);
+            Database database = catalog.getDbNullable(dbId);
             if (database == null) {
                 continue;
             }
@@ -647,36 +640,32 @@ public class ShowExecutor {
     private void handleShowTable() throws AnalysisException {
         ShowTableStmt showTableStmt = (ShowTableStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showTableStmt.getDb());
-        if (db != null) {
-            Map<String, String> tableMap = Maps.newTreeMap();
-            PatternMatcher matcher = null;
-            if (showTableStmt.getPattern() != null) {
-                matcher = PatternMatcher.createMysqlPattern(showTableStmt.getPattern(),
-                        CaseSensibility.TABLE.getCaseSensibility());
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showTableStmt.getDb());
+        Map<String, String> tableMap = Maps.newTreeMap();
+        PatternMatcher matcher = null;
+        if (showTableStmt.getPattern() != null) {
+            matcher = PatternMatcher.createMysqlPattern(showTableStmt.getPattern(),
+                    CaseSensibility.TABLE.getCaseSensibility());
+        }
+        for (Table tbl : db.getTables()) {
+            if (matcher != null && !matcher.match(tbl.getName())) {
+                continue;
             }
-            for (Table tbl : db.getTables()) {
-                if (matcher != null && !matcher.match(tbl.getName())) {
-                    continue;
-                }
-                // check tbl privs
-                if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(),
-                        db.getFullName(), tbl.getName(),
-                        PrivPredicate.SHOW)) {
-                    continue;
-                }
-                tableMap.put(tbl.getName(), tbl.getMysqlType());
+            // check tbl privs
+            if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(),
+                    db.getFullName(), tbl.getName(),
+                    PrivPredicate.SHOW)) {
+                continue;
             }
+            tableMap.put(tbl.getName(), tbl.getMysqlType());
+        }
 
-            for (Map.Entry<String, String> entry : tableMap.entrySet()) {
-                if (showTableStmt.isVerbose()) {
-                    rows.add(Lists.newArrayList(entry.getKey(), entry.getValue()));
-                } else {
-                    rows.add(Lists.newArrayList(entry.getKey()));
-                }
+        for (Map.Entry<String, String> entry : tableMap.entrySet()) {
+            if (showTableStmt.isVerbose()) {
+                rows.add(Lists.newArrayList(entry.getKey(), entry.getValue()));
+            } else {
+                rows.add(Lists.newArrayList(entry.getKey()));
             }
-        } else {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showTableStmt.getDb());
         }
         resultSet = new ShowResultSet(showTableStmt.getMetaData(), rows);
     }
@@ -685,7 +674,7 @@ public class ShowExecutor {
     private void handleShowTableStatus() throws AnalysisException {
         ShowTableStatusStmt showStmt = (ShowTableStatusStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showStmt.getDb());
+        Database db = ctx.getCatalog().getDbNullable(showStmt.getDb());
         if (db != null) {
             PatternMatcher matcher = null;
             if (showStmt.getPattern() != null) {
@@ -770,10 +759,7 @@ public class ShowExecutor {
     private void handleShowCreateDb() throws AnalysisException {
         ShowCreateDbStmt showStmt = (ShowCreateDbStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showStmt.getDb());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDb());
-        }
+        ctx.getCatalog().getDbOrAnalysisException(showStmt.getDb());
         StringBuilder sb = new StringBuilder();
         sb.append("CREATE DATABASE `").append(ClusterNamespace.getNameFromFullName(showStmt.getDb())).append("`");
         rows.add(Lists.newArrayList(ClusterNamespace.getNameFromFullName(showStmt.getDb()), sb.toString()));
@@ -783,15 +769,9 @@ public class ShowExecutor {
     // Show create table
     private void handleShowCreateTable() throws AnalysisException {
         ShowCreateTableStmt showStmt = (ShowCreateTableStmt) stmt;
-        Database db = ctx.getCatalog().getDb(showStmt.getDb());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDb());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDb());
+        Table table = db.getTableOrAnalysisException(showStmt.getTable());
         List<List<String>> rows = Lists.newArrayList();
-        Table table = db.getTable(showStmt.getTable());
-        if (table == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTable());
-        }
 
         table.readLock();
         try {
@@ -831,58 +811,50 @@ public class ShowExecutor {
     private void handleShowColumn() throws AnalysisException {
         ShowColumnStmt showStmt = (ShowColumnStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showStmt.getDb());
-        if (db != null) {
-            Table table = db.getTable(showStmt.getTable());
-            if (table != null) {
-                PatternMatcher matcher = null;
-                if (showStmt.getPattern() != null) {
-                    matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(),
-                            CaseSensibility.COLUMN.getCaseSensibility());
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDb());
+        Table table = db.getTableOrAnalysisException(showStmt.getTable());
+        PatternMatcher matcher = null;
+        if (showStmt.getPattern() != null) {
+            matcher = PatternMatcher.createMysqlPattern(showStmt.getPattern(),
+                    CaseSensibility.COLUMN.getCaseSensibility());
+        }
+        table.readLock();
+        try {
+            List<Column> columns = table.getBaseSchema();
+            for (Column col : columns) {
+                if (matcher != null && !matcher.match(col.getName())) {
+                    continue;
                 }
-                table.readLock();
-                try {
-                    List<Column> columns = table.getBaseSchema();
-                    for (Column col : columns) {
-                        if (matcher != null && !matcher.match(col.getName())) {
-                            continue;
-                        }
-                        final String columnName = col.getName();
-                        final String columnType = col.getOriginType().toString();
-                        final String isAllowNull = col.isAllowNull() ? "YES" : "NO";
-                        final String isKey = col.isKey() ? "YES" : "NO";
-                        final String defaultValue = col.getDefaultValue();
-                        final String aggType = col.getAggregationType() == null ? "" : col.getAggregationType().toSql();
-                        if (showStmt.isVerbose()) {
-                            // Field Type Collation Null Key Default Extra
-                            // Privileges Comment
-                            rows.add(Lists.newArrayList(columnName,
-                                    columnType,
-                                    "",
-                                    isAllowNull,
-                                    isKey,
-                                    defaultValue,
-                                    aggType,
-                                    "",
-                                    col.getComment()));
-                        } else {
-                            // Field Type Null Key Default Extra
-                            rows.add(Lists.newArrayList(columnName,
-                                    columnType,
-                                    isAllowNull,
-                                    isKey,
-                                    defaultValue,
-                                    aggType));
-                        }
-                    }
-                } finally {
-                    table.readUnlock();
+                final String columnName = col.getName();
+                final String columnType = col.getOriginType().toString();
+                final String isAllowNull = col.isAllowNull() ? "YES" : "NO";
+                final String isKey = col.isKey() ? "YES" : "NO";
+                final String defaultValue = col.getDefaultValue();
+                final String aggType = col.getAggregationType() == null ? "" : col.getAggregationType().toSql();
+                if (showStmt.isVerbose()) {
+                    // Field Type Collation Null Key Default Extra
+                    // Privileges Comment
+                    rows.add(Lists.newArrayList(columnName,
+                            columnType,
+                            "",
+                            isAllowNull,
+                            isKey,
+                            defaultValue,
+                            aggType,
+                            "",
+                            col.getComment()));
+                } else {
+                    // Field Type Null Key Default Extra
+                    rows.add(Lists.newArrayList(columnName,
+                            columnType,
+                            isAllowNull,
+                            isKey,
+                            defaultValue,
+                            aggType));
                 }
-            } else {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, db.getFullName() + "." + showStmt.getTable());
             }
-        } else {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getDb() + "." + showStmt.getTable());
+        } finally {
+            table.readUnlock();
         }
         resultSet = new ShowResultSet(showStmt.getMetaData(), rows);
     }
@@ -891,27 +863,19 @@ public class ShowExecutor {
     private void handleShowIndex() throws AnalysisException {
         ShowIndexStmt showStmt = (ShowIndexStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTableName().toString());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDbName());
 
-        Table table = db.getTable(showStmt.getTableName().getTbl());
-        if (table != null && table instanceof OlapTable) {
-            table.readLock();
-            try {
-                List<Index> indexes = ((OlapTable) table).getIndexes();
-                for (Index index : indexes) {
-                    rows.add(Lists.newArrayList(showStmt.getTableName().toString(), "", index.getIndexName(),
-                            "", index.getColumns().stream().collect(Collectors.joining(",")), "", "", "", "",
-                            "", index.getIndexType().name(), index.getComment()));
-                }
-            } finally {
-                table.readUnlock();
+        OlapTable table = db.getOlapTableOrAnalysisException(showStmt.getTableName().getTbl());
+        table.readLock();
+        try {
+            List<Index> indexes = table.getIndexes();
+            for (Index index : indexes) {
+                rows.add(Lists.newArrayList(showStmt.getTableName().toString(), "", index.getIndexName(),
+                        "", String.join(",", index.getColumns()), "", "", "", "",
+                        "", index.getIndexType().name(), index.getComment()));
             }
-        } else {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR,
-                    db.getFullName() + "." + showStmt.getTableName().toString());
+        } finally {
+            table.readUnlock();
         }
         resultSet = new ShowResultSet(showStmt.getMetaData(), rows);
     }
@@ -999,10 +963,7 @@ public class ShowExecutor {
         ShowLoadStmt showStmt = (ShowLoadStmt) stmt;
 
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showStmt.getDbName());
         long dbId = db.getId();
 
         // combine the List<LoadInfo> of load(v1) and loadManager(v2)
@@ -1066,10 +1027,7 @@ public class ShowExecutor {
         ShowStreamLoadStmt showStmt = (ShowStreamLoadStmt) stmt;
 
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showStmt.getDbName());
         long dbId = db.getId();
 
         List<List<Comparable>> streamLoadRecords = catalog.getStreamLoadRecordMgr().getStreamLoadRecordByDb(dbId, showStmt.getLabelValue(), showStmt.isAccurateMatch(), showStmt.getState());
@@ -1123,10 +1081,7 @@ public class ShowExecutor {
         }
 
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showWarningsStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showWarningsStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showWarningsStmt.getDbName());
 
         long dbId = db.getId();
         Load load = catalog.getLoadInstance();
@@ -1345,10 +1300,7 @@ public class ShowExecutor {
         ShowDeleteStmt showStmt = (ShowDeleteStmt) stmt;
 
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showStmt.getDbName());
         long dbId = db.getId();
 
         DeleteHandler deleteHandler = catalog.getDeleteHandler();
@@ -1436,13 +1388,13 @@ public class ShowExecutor {
 
             // check real meta
             do {
-                Database db = catalog.getDb(dbId);
+                Database db = catalog.getDbNullable(dbId);
                 if (db == null) {
                     isSync = false;
                     break;
                 }
                 dbName = db.getFullName();
-                Table table = db.getTable(tableId);
+                Table table = db.getTableNullable(tableId);
                 if (table == null || !(table instanceof OlapTable)) {
                     isSync = false;
                     break;
@@ -1498,22 +1450,11 @@ public class ShowExecutor {
                                         partitionId.toString(), indexId.toString(),
                                         isSync.toString(), detailCmd));
         } else {
-            Database db = catalog.getDb(showStmt.getDbName());
-            if (db == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-            }
+            Database db = catalog.getDbOrAnalysisException(showStmt.getDbName());
+            OlapTable olapTable = db.getOlapTableOrAnalysisException(showStmt.getTableName());
 
-            Table table = db.getTable(showStmt.getTableName());
-            if (table == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getTableName());
-            }
-            if (!(table instanceof OlapTable)) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_NOT_OLAP_TABLE, showStmt.getTableName());
-            }
-
-            table.readLock();
+            olapTable.readLock();
             try {
-                OlapTable olapTable = (OlapTable) table;
                 long sizeLimit = -1;
                 if (showStmt.hasOffset() && showStmt.hasLimit()) {
                     sizeLimit = showStmt.getOffset() + showStmt.getLimit();
@@ -1553,7 +1494,7 @@ public class ShowExecutor {
                         if (indexId > -1 && index.getId() != indexId) {
                             continue;
                         }
-                        TabletsProcDir procDir = new TabletsProcDir(table, index);
+                        TabletsProcDir procDir = new TabletsProcDir(olapTable, index);
                         tabletInfos.addAll(procDir.fetchComparableResult(
                                 showStmt.getVersion(), showStmt.getBackendId(), showStmt.getReplicaState()));
                         if (sizeLimit > -1 && tabletInfos.size() >= sizeLimit) {
@@ -1588,7 +1529,7 @@ public class ShowExecutor {
                     rows.add(oneTablet);
                 }
             } finally {
-                table.readUnlock();
+                olapTable.readUnlock();
             }
         }
 
@@ -1657,10 +1598,7 @@ public class ShowExecutor {
     private void handleShowExport() throws AnalysisException {
         ShowExportStmt showExportStmt = (ShowExportStmt) stmt;
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showExportStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showExportStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showExportStmt.getDbName());
         long dbId = db.getId();
 
         ExportMgr exportMgr = catalog.getExportMgr();
@@ -1718,10 +1656,7 @@ public class ShowExecutor {
 
     private void handleShowBackup() throws AnalysisException {
         ShowBackupStmt showStmt = (ShowBackupStmt) stmt;
-        Database db = Catalog.getCurrentCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(showStmt.getDbName());
 
         List<AbstractJob> jobs = Catalog.getCurrentCatalog().getBackupHandler().getJobs(db.getId(), showStmt.getLabelPredicate());
 
@@ -1735,10 +1670,7 @@ public class ShowExecutor {
 
     private void handleShowRestore() throws AnalysisException {
         ShowRestoreStmt showStmt = (ShowRestoreStmt) stmt;
-        Database db = Catalog.getCurrentCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(showStmt.getDbName());
 
         List<AbstractJob> jobs = Catalog.getCurrentCatalog().getBackupHandler().getJobs(db.getId(), showStmt.getLabelPredicate());
 
@@ -1753,10 +1685,7 @@ public class ShowExecutor {
     private void handleShowSyncJobs() throws AnalysisException {
         ShowSyncJobStmt showStmt = (ShowSyncJobStmt) stmt;
         Catalog catalog = Catalog.getCurrentCatalog();
-        Database db = catalog.getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = catalog.getDbOrAnalysisException(showStmt.getDbName());
 
         List<List<Comparable>> syncInfos = catalog.getSyncJobManager().getSyncJobsInfoByDbId(db.getId());
         Collections.sort(syncInfos, new ListComparator<List<Comparable>>(0));
@@ -1858,7 +1787,7 @@ public class ShowExecutor {
     private void handleShowDynamicPartition() {
         ShowDynamicPartitionStmt showDynamicPartitionStmt = (ShowDynamicPartitionStmt) stmt;
         List<List<String>> rows = Lists.newArrayList();
-        Database db = ctx.getCatalog().getDb(showDynamicPartitionStmt.getDb());
+        Database db = ctx.getCatalog().getDbNullable(showDynamicPartitionStmt.getDb());
         if (db != null) {
             List<Table> tableList = db.getTables();
             for (Table tbl : tableList) {
@@ -1912,10 +1841,7 @@ public class ShowExecutor {
     // Show transaction statement.
     private void handleShowTransaction() throws AnalysisException {
         ShowTransactionStmt showStmt = (ShowTransactionStmt) stmt;
-        Database db = ctx.getCatalog().getDb(showStmt.getDbName());
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, showStmt.getDbName());
-        }
+        Database db = ctx.getCatalog().getDbOrAnalysisException(showStmt.getDbName());
 
         Long txnId = showStmt.getTxnId();
         String label = showStmt.getLabel();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index 75d1c1e..742672c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -1059,14 +1059,8 @@ public class StmtExecutor implements ProfileWriter {
         TTxnParams txnConf = txnEntry.getTxnConf();
         long timeoutSecond = ConnectContext.get().getSessionVariable().getQueryTimeoutS();
         TransactionState.LoadJobSourceType sourceType = TransactionState.LoadJobSourceType.INSERT_STREAMING;
-        Database dbObj = Catalog.getCurrentCatalog().getDb(dbName);
-        if (dbObj == null) {
-            throw new TException("database is invalid for dbName: " + dbName);
-        }
-        Table tblObj = dbObj.getTable(tblName);
-        if (tblObj == null) {
-            throw new TException("table is invalid: " + tblName);
-        }
+        Database dbObj = Catalog.getCurrentCatalog().getDbOrException(dbName, s -> new TException("database is invalid for dbName: " + s));
+        Table tblObj = dbObj.getTableOrException(tblName, s -> new TException("table is invalid: " + s));
         txnConf.setDbId(dbObj.getId()).setTbl(tblName).setDb(dbName);
         txnEntry.setTable(tblObj);
         txnEntry.setDb(dbObj);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index a25f29f..81955de 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -204,13 +204,13 @@ public class FrontendServiceImpl implements FrontendService.Iface {
 
         // database privs should be checked in analysis phrase
 
-        Database db = Catalog.getCurrentCatalog().getDb(params.db);
-        UserIdentity currentUser = null;
+        UserIdentity currentUser;
         if (params.isSetCurrentUserIdent()) {
             currentUser = UserIdentity.fromThrift(params.current_user_ident);
         } else {
             currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip);
         }
+        Database db = Catalog.getCurrentCatalog().getDbNullable(params.db);
         if (db != null) {
             for (String tableName : db.getTableNamesWithLock()) {
                 LOG.debug("get table: {}, wait to check", tableName);
@@ -246,13 +246,13 @@ public class FrontendServiceImpl implements FrontendService.Iface {
 
         // database privs should be checked in analysis phrase
 
-        Database db = Catalog.getCurrentCatalog().getDb(params.db);
-        UserIdentity currentUser = null;
+        UserIdentity currentUser;
         if (params.isSetCurrentUserIdent()) {
             currentUser = UserIdentity.fromThrift(params.current_user_ident);
         } else {
             currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip);
         }
+        Database db = Catalog.getCurrentCatalog().getDbNullable(params.db);
         if (db != null) {
             List<Table> tables = null;
             if (!params.isSetType() || params.getType() == null || params.getType().isEmpty()) {
@@ -381,9 +381,9 @@ public class FrontendServiceImpl implements FrontendService.Iface {
             return result;
         }
 
-        Database db = Catalog.getCurrentCatalog().getDb(params.db);
+        Database db = Catalog.getCurrentCatalog().getDbNullable(params.db);
         if (db != null) {
-            Table table = db.getTable(params.getTableName());
+            Table table = db.getTableNullable(params.getTableName());
             if (table != null) {
                 table.readLock();
                 try {
@@ -668,10 +668,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
 
 
     private void checkAuthCodeUuid(String dbName, long txnId, String authCodeUuid) throws AuthenticationException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            throw new AuthenticationException("invalid db name: " + dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrException(dbName, s -> new AuthenticationException("invalid db name: " + s));
         TransactionState transactionState = Catalog.getCurrentGlobalTransactionMgr().
                 getTransactionState(db.getId(), txnId);
         if (transactionState == null) {
@@ -779,7 +776,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         // check database
         Catalog catalog = Catalog.getCurrentCatalog();
         String fullDbName = ClusterNamespace.getFullName(cluster, request.getDb());
-        Database db = catalog.getDb(fullDbName);
+        Database db = catalog.getDbNullable(fullDbName);
         if (db == null) {
             String dbName = fullDbName;
             if (Strings.isNullOrEmpty(request.getCluster())) {
@@ -788,7 +785,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
             throw new UserException("unknown database, database=" + dbName);
         }
 
-        Table table = db.getTableOrThrowException(request.tbl, TableType.OLAP);
+        Table table = db.getTableOrMetaException(request.tbl, TableType.OLAP);
         // begin
         long timeoutSecond = request.isSetTimeout() ? request.getTimeout() : Config.stream_load_default_timeout_second;
         MetricRepo.COUNTER_LOAD_ADD.increase(1L);
@@ -851,11 +848,11 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         // get database
         Catalog catalog = Catalog.getCurrentCatalog();
         String fullDbName = ClusterNamespace.getFullName(cluster, request.getDb());
-        Database db = null;
+        Database db;
         if (request.isSetDbId() && request.getDbId() > 0) {
-            db = catalog.getDb(request.getDbId());
+            db = catalog.getDbNullable(request.getDbId());
         } else {
-            db = catalog.getDb(fullDbName);
+            db = catalog.getDbNullable(fullDbName);
         }
         if (db == null) {
             String dbName = fullDbName;
@@ -866,7 +863,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         }
 
         long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() / 2 : 5000;
-        Table table = db.getTableOrThrowException(request.getTbl(), TableType.OLAP);
+        Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP);
         boolean ret = Catalog.getCurrentGlobalTransactionMgr().commitAndPublishTransaction(
                         db, Lists.newArrayList(table), request.getTxnId(),
                         TabletCommitInfo.fromThrift(request.getCommitInfos()),
@@ -916,11 +913,11 @@ public class FrontendServiceImpl implements FrontendService.Iface {
                     request.getTbl(), request.getUserIp(), PrivPredicate.LOAD);
         }
         String dbName = ClusterNamespace.getFullName(cluster, request.getDb());
-        Database db = null;
+        Database db;
         if (request.isSetDbId() && request.getDbId() > 0) {
-            db = Catalog.getCurrentCatalog().getDb(request.getDbId());
+            db = Catalog.getCurrentCatalog().getDbNullable(request.getDbId());
         } else {
-            db = Catalog.getCurrentCatalog().getDb(dbName);
+            db = Catalog.getCurrentCatalog().getDbNullable(dbName);
         }
         if (db == null) {
             throw new MetaNotFoundException("db " + request.getDb() + " does not exist");
@@ -962,7 +959,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
 
         Catalog catalog = Catalog.getCurrentCatalog();
         String fullDbName = ClusterNamespace.getFullName(cluster, request.getDb());
-        Database db = catalog.getDb(fullDbName);
+        Database db = catalog.getDbNullable(fullDbName);
         if (db == null) {
             String dbName = fullDbName;
             if (Strings.isNullOrEmpty(request.getCluster())) {
@@ -971,7 +968,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
             throw new UserException("unknown database, database=" + dbName);
         }
         long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000;
-        Table table = db.getTableOrThrowException(request.getTbl(), TableType.OLAP);
+        Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP);
         if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) {
             throw new UserException("get table read lock timeout, database=" + fullDbName + ",table=" + table.getName());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java
index 0207122..e34366f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/StatisticsManager.java
@@ -61,17 +61,10 @@ public class StatisticsManager {
 
     public List<List<String>> showTableStatsList(String dbName, String tableName)
             throws AnalysisException {
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
         List<List<String>> result = Lists.newArrayList();
         if (tableName != null) {
-            Table table = db.getTable(tableName);
-            // check meta
-            if (table == null) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-            }
+            Table table = db.getTableOrAnalysisException(tableName);
             // check priv
             if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName, tableName,
                     PrivPredicate.SHOW)) {
@@ -139,14 +132,8 @@ public class StatisticsManager {
         String dbName = dbTableName.getDb();
         String tableName = dbTableName.getTbl();
 
-        Database db = Catalog.getCurrentCatalog().getDb(dbName);
-        if (db == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_DB_ERROR, dbName);
-        }
-        Table table = db.getTable(tableName);
-        if (table == null) {
-            ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
-        }
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbName);
+        Table table = db.getTableOrAnalysisException(tableName);
         return table;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
index bc5a9ff..5e5fc15 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
@@ -897,9 +897,9 @@ public class SystemInfoService {
     }
 
     public void updateBackendReportVersion(long backendId, long newReportVersion, long dbId, long tableId) {
-        AtomicLong atomicLong = null;
+        AtomicLong atomicLong;
         if ((atomicLong = idToReportVersionRef.get(backendId)) != null) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 LOG.warn("failed to update backend report version, db {} does not exist", dbId);
                 return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/ExportPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/ExportPendingTask.java
index c73217c..ad90778 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/ExportPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/ExportPendingTask.java
@@ -58,7 +58,7 @@ public class ExportPendingTask extends MasterTask {
         }
 
         long dbId = job.getDbId();
-        db = Catalog.getCurrentCatalog().getDb(dbId);
+        db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             job.cancel(ExportFailMsg.CancelType.RUN_FAIL, "database does not exist");
             return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
index 7f5fc54..cffd772 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
@@ -95,10 +95,7 @@ public class HadoopLoadPendingTask extends LoadPendingTask {
             throw new LoadException("txn does not exist: " + job.getTransactionId());
         }
         for (long tableId : job.getIdToTableLoadInfo().keySet()) {
-            OlapTable table = (OlapTable) db.getTable(tableId);
-            if (table == null) {
-                throw new LoadException("table does not exist. id: " + tableId);
-            }
+            OlapTable table = (OlapTable) db.getTableOrException(tableId, s -> new LoadException("table does not exist. id: " + s));
             table.readLock();
             try {
                 txnState.addTableIndexes(table);
@@ -132,10 +129,7 @@ public class HadoopLoadPendingTask extends LoadPendingTask {
             long tableId = tableEntry.getKey();
             TableLoadInfo tableLoadInfo = tableEntry.getValue();
 
-            OlapTable table = (OlapTable) db.getTable(tableId);
-            if (table == null) {
-                throw new LoadException("table does not exist. id: " + tableId);
-            }
+            OlapTable table = (OlapTable) db.getTableOrException(tableId, s -> new LoadException("table does not exist. id: " + s));
             table.readLock();
             try {
                 // columns
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java
index 7262e2e..43b9a9c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/LoadEtlTask.java
@@ -84,7 +84,7 @@ public abstract class LoadEtlTask extends MasterTask {
         
         // check db
         long dbId = job.getDbId();
-        db = Catalog.getCurrentCatalog().getDb(dbId);
+        db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             load.cancelLoadJob(job, CancelType.ETL_RUN_FAIL, "db does not exist. id: " + dbId);
             return;
@@ -160,10 +160,7 @@ public abstract class LoadEtlTask extends MasterTask {
         try {
             for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
                 long tableId = tableEntry.getKey();
-                OlapTable table = (OlapTable) db.getTable(tableId);
-                if (table == null) {
-                    throw new MetaNotFoundException("table does not exist. id: " + tableId);
-                }
+                OlapTable table = (OlapTable) db.getTableOrMetaException(tableId);
                 
                 TableLoadInfo tableLoadInfo = tableEntry.getValue();
                 Map<Long, PartitionLoadInfo> idToPartitionLoadInfo = tableLoadInfo.getIdToPartitionLoadInfo();
@@ -233,10 +230,7 @@ public abstract class LoadEtlTask extends MasterTask {
         Map<Long, TableLoadInfo> idToTableLoadInfo = job.getIdToTableLoadInfo();
         for (Entry<Long, TableLoadInfo> tableEntry : idToTableLoadInfo.entrySet()) {
             long tableId = tableEntry.getKey();
-            OlapTable table = (OlapTable) db.getTable(tableId);
-            if (table == null) {
-                throw new LoadException("table does not exist. id: " + tableId);
-            }
+            OlapTable table = (OlapTable) db.getTableOrException(tableId, s -> new LoadException("table does not exist. id: " + s));
 
             table.readLock();
             try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java
index 5242ca0..0d2f699 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/LoadPendingTask.java
@@ -69,7 +69,7 @@ public abstract class LoadPendingTask extends MasterTask {
         
         // get db
         long dbId = job.getDbId();
-        db = Catalog.getCurrentCatalog().getDb(dbId);
+        db = Catalog.getCurrentCatalog().getDbNullable(dbId);
         if (db == null) {
             load.cancelLoadJob(job, CancelType.ETL_SUBMIT_FAIL, "db does not exist. id: " + dbId);
             return;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
index 23b42ee..ee04b40 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
@@ -327,10 +327,7 @@ public class DatabaseTransactionMgr {
 
 
     private void checkDatabaseDataQuota() throws MetaNotFoundException, QuotaExceedException {
-        Database db = catalog.getDb(dbId);
-        if (db == null) {
-            throw new MetaNotFoundException("Database[" + dbId + "] does not exist");
-        }
+        Database db = catalog.getDbOrMetaException(dbId);
 
         if (usedQuotaDataBytes == -1) {
             usedQuotaDataBytes = db.getUsedDataQuotaWithLock();
@@ -360,12 +357,8 @@ public class DatabaseTransactionMgr {
             throws UserException {
         // 1. check status
         // the caller method already own db lock, we do not obtain db lock here
-        Database db = catalog.getDb(dbId);
-        if (null == db) {
-            throw new MetaNotFoundException("could not find db [" + dbId + "]");
-        }
-
-        TransactionState transactionState = null;
+        Database db = catalog.getDbOrMetaException(dbId);
+        TransactionState transactionState;
         readLock();
         try {
             transactionState = unprotectedGetTransactionState(transactionId);
@@ -455,10 +448,7 @@ public class DatabaseTransactionMgr {
         Set<Long> errorReplicaIds = Sets.newHashSet();
         Set<Long> totalInvolvedBackends = Sets.newHashSet();
         for (long tableId : tableToPartition.keySet()) {
-            OlapTable table = (OlapTable) db.getTable(tableId);
-            if (table == null) {
-                throw new MetaNotFoundException("Table does not exist: " + tableId);
-            }
+            OlapTable table = (OlapTable) db.getTableOrMetaException(tableId);
             for (Partition partition : table.getAllPartitions()) {
                 if (!tableToPartition.get(tableId).contains(partition.getId())) {
                     continue;
@@ -698,7 +688,7 @@ public class DatabaseTransactionMgr {
             errorReplicaIds.addAll(originalErrorReplicas);
         }
 
-        Database db = catalog.getDb(transactionState.getDbId());
+        Database db = catalog.getDbNullable(transactionState.getDbId());
         if (db == null) {
             writeLock();
             try {
@@ -735,7 +725,7 @@ public class DatabaseTransactionMgr {
             while (tableCommitInfoIterator.hasNext()) {
                 TableCommitInfo tableCommitInfo = tableCommitInfoIterator.next();
                 long tableId = tableCommitInfo.getTableId();
-                OlapTable table = (OlapTable) db.getTable(tableId);
+                OlapTable table = (OlapTable) db.getTableNullable(tableId);
                 // table maybe dropped between commit and publish, ignore this error
                 if (table == null) {
                     tableCommitInfoIterator.remove();
@@ -888,7 +878,7 @@ public class DatabaseTransactionMgr {
         for (long tableId : tableToPartition.keySet()) {
             TableCommitInfo tableCommitInfo = new TableCommitInfo(tableId);
             for (long partitionId : tableToPartition.get(tableId)) {
-                OlapTable table = (OlapTable) db.getTable(tableId);
+                OlapTable table = (OlapTable) db.getTableNullable(tableId);
                 Partition partition = table.getPartition(partitionId);
                 PartitionCommitInfo partitionCommitInfo = new PartitionCommitInfo(partitionId,
                         partition.getNextVersion(), partition.getNextVersionHash(),
@@ -1244,11 +1234,7 @@ public class DatabaseTransactionMgr {
         List<List<String>> infos = new ArrayList<List<String>>();
         readLock();
         try {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
-            if (db == null) {
-                throw new AnalysisException("Database[" + dbId + "] does not exist");
-            }
-
+            Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbId);
             TransactionState txnState = unprotectedGetTransactionState(txnId);
             if (txnState == null) {
                 throw new AnalysisException("transaction with id " + txnId + " does not exist");
@@ -1258,7 +1244,7 @@ public class DatabaseTransactionMgr {
                 // check auth
                 Set<Long> tblIds = txnState.getIdToTableCommitInfos().keySet();
                 for (Long tblId : tblIds) {
-                    Table tbl = db.getTable(tblId);
+                    Table tbl = db.getTableNullable(tblId);
                     if (tbl != null) {
                         if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), db.getFullName(),
                                 tbl.getName(), PrivPredicate.SHOW)) {
@@ -1302,7 +1288,7 @@ public class DatabaseTransactionMgr {
         Set<Long> errorReplicaIds = transactionState.getErrorReplicas();
         for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) {
             long tableId = tableCommitInfo.getTableId();
-            OlapTable table = (OlapTable) db.getTable(tableId);
+            OlapTable table = (OlapTable) db.getTableNullable(tableId);
             if (table == null) {
                 LOG.warn("table {} does not exist when update catalog after committed. transaction: {}, db: {}",
                         tableId, transactionState.getTransactionId(), db.getId());
@@ -1342,7 +1328,7 @@ public class DatabaseTransactionMgr {
         Set<Long> errorReplicaIds = transactionState.getErrorReplicas();
         for (TableCommitInfo tableCommitInfo : transactionState.getIdToTableCommitInfos().values()) {
             long tableId = tableCommitInfo.getTableId();
-            OlapTable table = (OlapTable) db.getTable(tableId);
+            OlapTable table = (OlapTable) db.getTableNullable(tableId);
             if (table == null) {
                 LOG.warn("table {} does not exist when update catalog after visible. transaction: {}, db: {}",
                         tableId, transactionState.getTransactionId(), db.getId());
@@ -1485,12 +1471,12 @@ public class DatabaseTransactionMgr {
         }
     }
 
-    public void replayUpsertTransactionState(TransactionState transactionState) {
+    public void replayUpsertTransactionState(TransactionState transactionState) throws MetaNotFoundException {
         writeLock();
         try {
             // set transaction status will call txn state change listener
             transactionState.replaySetTransactionStatus();
-            Database db = catalog.getDb(transactionState.getDbId());
+            Database db = catalog.getDbOrMetaException(transactionState.getDbId());
             if (transactionState.getTransactionStatus() == TransactionStatus.COMMITTED) {
                 LOG.info("replay a committed transaction {}", transactionState);
                 updateCatalogAfterCommitted(transactionState, db);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java
index f62770b..22b9a27 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DbUsedDataQuotaInfoCollector.java
@@ -44,7 +44,7 @@ public class DbUsedDataQuotaInfoCollector extends MasterDaemon {
         List<Long> dbIdList = catalog.getDbIds();
         GlobalTransactionMgr globalTransactionMgr = catalog.getGlobalTransactionMgr();
         for (Long dbId : dbIdList) {
-            Database db = catalog.getDb(dbId);
+            Database db = catalog.getDbNullable(dbId);
             if (db == null) {
                 LOG.warn("Database [" + dbId + "] does not exist, skip to update database used data quota");
                 continue;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
index 83ebf07..e3416d2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/GlobalTransactionMgr.java
@@ -337,23 +337,23 @@ public class GlobalTransactionMgr implements Writable {
 
     // for replay idToTransactionState
     // check point also run transaction cleaner, the cleaner maybe concurrently modify id to 
-    public void replayUpsertTransactionState(TransactionState transactionState) {
+    public void replayUpsertTransactionState(TransactionState transactionState) throws MetaNotFoundException {
         try {
             DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId());
             dbTransactionMgr.replayUpsertTransactionState(transactionState);
         } catch (AnalysisException e) {
-            LOG.warn("replay upsert transaction [" + transactionState.getTransactionId() + "] failed", e);
+            throw new MetaNotFoundException(e);
         }
     }
 
     @Deprecated
     // Use replayBatchDeleteTransactions instead
-    public void replayDeleteTransactionState(TransactionState transactionState) {
+    public void replayDeleteTransactionState(TransactionState transactionState) throws MetaNotFoundException {
         try {
             DatabaseTransactionMgr dbTransactionMgr = getDatabaseTransactionMgr(transactionState.getDbId());
             dbTransactionMgr.replayDeleteTransaction(transactionState);
         } catch (AnalysisException e) {
-            LOG.warn("replay delete transaction [" + transactionState.getTransactionId() + "] failed", e);
+            throw new MetaNotFoundException(e);
         }
     }
 
@@ -375,7 +375,7 @@ public class GlobalTransactionMgr implements Writable {
         for (long dbId : dbIds) {
             List<Comparable> info = new ArrayList<Comparable>();
             info.add(dbId);
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
+            Database db = Catalog.getCurrentCatalog().getDbNullable(dbId);
             if (db == null) {
                 continue;
             }
@@ -514,10 +514,7 @@ public class GlobalTransactionMgr implements Writable {
         long dbId = request.getDbId();
         int commitTimeoutSec = Config.commit_timeout_second;
         for (int i = 0; i < commitTimeoutSec; ++i) {
-            Database db = Catalog.getCurrentCatalog().getDb(dbId);
-            if (db == null) {
-                throw new AnalysisException("invalid db id: " + dbId);
-            }
+            Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException(dbId);
             TWaitingTxnStatusResult statusResult = new TWaitingTxnStatusResult();
             statusResult.status = new TStatus();
             TransactionStatus txnStatus = null;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java
index 291ab88..cd7d663 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/PublishVersionDaemon.java
@@ -185,7 +185,7 @@ public class PublishVersionDaemon extends MasterDaemon {
                             continue;
                         }
 
-                        Database db = Catalog.getCurrentCatalog().getDb(transactionState.getDbId());
+                        Database db = Catalog.getCurrentCatalog().getDbNullable(transactionState.getDbId());
                         if (db == null) {
                             LOG.warn("Database [{}] has been dropped.", transactionState.getDbId());
                             continue;
@@ -194,7 +194,7 @@ public class PublishVersionDaemon extends MasterDaemon {
 
                         for (int i = 0; i < transactionState.getTableIdList().size(); i++) {
                             long tableId = transactionState.getTableIdList().get(i);
-                            Table table = db.getTable(tableId);
+                            Table table = db.getTableNullable(tableId);
                             if (table == null || table.getType() != Table.TableType.OLAP) {
                                 LOG.warn("Table [{}] in database [{}] has been dropped.", tableId, db.getFullName());
                                 continue;
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
index 1b28317..3c524a1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
@@ -24,6 +24,7 @@ import org.apache.doris.analysis.ShowAlterStmt;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
@@ -97,7 +98,7 @@ public class AlterJobV2Test {
         System.out.println(showResultSet.getResultRows());
     }
 
-    private void waitAlterJobDone(Map<Long, AlterJobV2> alterJobs) throws InterruptedException {
+    private void waitAlterJobDone(Map<Long, AlterJobV2> alterJobs) throws Exception {
         for (AlterJobV2 alterJobV2 : alterJobs.values()) {
             while (!alterJobV2.getJobState().isFinalState()) {
                 System.out.println("alter job " + alterJobV2.getDbId() + " is running. state: " + alterJobV2.getJobState());
@@ -106,8 +107,8 @@ public class AlterJobV2Test {
             System.out.println("alter job " + alterJobV2.getDbId() + " is done. state: " + alterJobV2.getJobState());
             Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
 
-            Database db = Catalog.getCurrentCatalog().getDb(alterJobV2.getDbId());
-            OlapTable tbl = (OlapTable) db.getTable(alterJobV2.getTableId());
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(alterJobV2.getDbId());
+            OlapTable tbl = db.getTableOrMetaException(alterJobV2.getTableId(), Table.TableType.OLAP);
             while (tbl.getState() != OlapTable.OlapTableState.NORMAL) {
                 Thread.sleep(1000);
             }
@@ -136,10 +137,8 @@ public class AlterJobV2Test {
     @Deprecated
     public void testAlterSegmentV2() throws Exception {
         // TODO this test should remove after we disable segment v1 completely
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        Assert.assertNotNull(db);
-        OlapTable tbl = (OlapTable) db.getTable("segmentv2");
-        Assert.assertNotNull(tbl);
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = db.getTableOrMetaException("segmentv2", Table.TableType.OLAP);
         Assert.assertEquals(TStorageFormat.V1, tbl.getTableProperty().getStorageFormat());
 
         // 1. create a rollup r1
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index a206165..2a77d2f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -212,8 +212,8 @@ public class AlterTest {
 
     @Test
     public void alterTableModifyComment() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        Table tbl = db.getTable("tbl5");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        Table tbl = db.getTableOrMetaException("tbl5");
 
         // table comment
         String stmt = "alter table test.tbl5 modify comment 'comment1'";
@@ -292,8 +292,8 @@ public class AlterTest {
                 "'dynamic_partition.buckets' = '3'\n" +
                 " );";
         alterTable(stmt, false);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable tbl = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("tbl1");
         Assert.assertTrue(tbl.getTableProperty().getDynamicPartitionProperty().getEnable());
         Assert.assertEquals(4, tbl.getIndexIdToSchema().size());
 
@@ -329,7 +329,7 @@ public class AlterTest {
         Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl.getPartitionInfo().getReplicationNum(p1.getId())));
 
         // set un-partitioned table's real replication num
-        OlapTable tbl2 = (OlapTable) db.getTable("tbl2");
+        OlapTable tbl2 = (OlapTable) db.getTableOrMetaException("tbl2");
         Partition partition = tbl2.getPartition(tbl2.getName());
         Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId())));
         stmt = "alter table test.tbl2 set ('replication_num' = '3');";
@@ -349,8 +349,8 @@ public class AlterTest {
     // test batch update range partitions' properties
     @Test
     public void testBatchUpdatePartitionProperties() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable tbl4 = (OlapTable) db.getTable("tbl4");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl4 = (OlapTable) db.getTableOrMetaException("tbl4");
         Partition p1 = tbl4.getPartition("p1");
         Partition p2 = tbl4.getPartition("p2");
         Partition p3 = tbl4.getPartition("p3");
@@ -419,14 +419,14 @@ public class AlterTest {
         alterTable(stmt, false);
         Thread.sleep(5000); // sleep to wait dynamic partition scheduler run
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable tbl = (OlapTable) db.getTable("tbl3");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("tbl3");
         Assert.assertEquals(4, tbl.getPartitionNames().size());
         Assert.assertNull(tbl.getPartition("p1"));
         Assert.assertNull(tbl.getPartition("p2"));
     }
 
-    private void waitSchemaChangeJobDone(boolean rollupJob) throws InterruptedException {
+    private void waitSchemaChangeJobDone(boolean rollupJob) throws Exception {
         Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2();
         if (rollupJob) {
             alterJobs = Catalog.getCurrentCatalog().getRollupHandler().getAlterJobsV2();
@@ -438,8 +438,8 @@ public class AlterTest {
             }
             System.out.println(alterJobV2.getType() + " alter job " + alterJobV2.getJobId() + " is done. state: " + alterJobV2.getJobState());
             Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
-            Database db = Catalog.getCurrentCatalog().getDb(alterJobV2.getDbId());
-            OlapTable tbl = (OlapTable) db.getTable(alterJobV2.getTableId());
+            Database db = Catalog.getCurrentCatalog().getDbOrMetaException(alterJobV2.getDbId());
+            OlapTable tbl = (OlapTable) db.getTableOrMetaException(alterJobV2.getTableId());
             while (tbl.getState() != OlapTable.OlapTableState.NORMAL) {
                 Thread.sleep(1000);
             }
@@ -578,14 +578,14 @@ public class AlterTest {
         createTable(stmt2);
         createTable(stmt3);
         createTable(stmt4);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
 
         // table name -> tabletIds
         Map<String, List<Long>> tblNameToTabletIds = Maps.newHashMap();
-        OlapTable replace1Tbl = (OlapTable) db.getTable("replace1");
-        OlapTable r1Tbl = (OlapTable) db.getTable("r1");
-        OlapTable replace2Tbl = (OlapTable) db.getTable("replace2");
-        OlapTable replace3Tbl = (OlapTable) db.getTable("replace3");
+        OlapTable replace1Tbl = (OlapTable) db.getTableOrMetaException("replace1");
+        OlapTable r1Tbl = (OlapTable) db.getTableOrMetaException("r1");
+        OlapTable replace2Tbl = (OlapTable) db.getTableOrMetaException("replace2");
+        OlapTable replace3Tbl = (OlapTable) db.getTableOrMetaException("replace3");
 
         tblNameToTabletIds.put("replace1", Lists.newArrayList());
         for (Partition partition : replace1Tbl.getAllPartitions()) {
@@ -629,8 +629,8 @@ public class AlterTest {
 
         // replace1 with replace2
         replaceStmt = "ALTER TABLE test.replace1 REPLACE WITH TABLE replace2";
-        OlapTable replace1 = (OlapTable) db.getTable("replace1");
-        OlapTable replace2 = (OlapTable) db.getTable("replace2");
+        OlapTable replace1 = (OlapTable) db.getTableOrMetaException("replace1");
+        OlapTable replace2 = (OlapTable) db.getTableOrMetaException("replace2");
         Assert.assertEquals(3, replace1.getPartition("replace1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertEquals(1, replace2.getPartition("replace2").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
 
@@ -638,8 +638,8 @@ public class AlterTest {
         Assert.assertTrue(checkAllTabletsExists(tblNameToTabletIds.get("replace1")));
         Assert.assertTrue(checkAllTabletsExists(tblNameToTabletIds.get("replace2")));
 
-        replace1 = (OlapTable) db.getTable("replace1");
-        replace2 = (OlapTable) db.getTable("replace2");
+        replace1 = (OlapTable) db.getTableOrMetaException("replace1");
+        replace2 = (OlapTable) db.getTableOrMetaException("replace2");
         Assert.assertEquals(1, replace1.getPartition("replace1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertEquals(3, replace2.getPartition("replace2").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertEquals("replace1", replace1.getIndexNameById(replace1.getBaseIndexId()));
@@ -648,8 +648,8 @@ public class AlterTest {
         // replace with no swap
         replaceStmt = "ALTER TABLE test.replace1 REPLACE WITH TABLE replace2 properties('swap' = 'false')";
         alterTable(replaceStmt, false);
-        replace1 = (OlapTable) db.getTable("replace1");
-        replace2 = (OlapTable) db.getTable("replace2");
+        replace1 = (OlapTable) db.getTableNullable("replace1");
+        replace2 = (OlapTable) db.getTableNullable("replace2");
         Assert.assertNull(replace2);
         Assert.assertEquals(3, replace1.getPartition("replace1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertEquals("replace1", replace1.getIndexNameById(replace1.getBaseIndexId()));
@@ -658,8 +658,8 @@ public class AlterTest {
 
         replaceStmt = "ALTER TABLE test.replace1 REPLACE WITH TABLE replace3 properties('swap' = 'true')";
         alterTable(replaceStmt, false);
-        replace1 = (OlapTable) db.getTable("replace1");
-        OlapTable replace3 = (OlapTable) db.getTable("replace3");
+        replace1 = (OlapTable) db.getTableOrMetaException("replace1");
+        OlapTable replace3 = (OlapTable) db.getTableOrMetaException("replace3");
         Assert.assertEquals(3, replace1.getPartition("p1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertEquals(3, replace1.getPartition("p2").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE).size());
         Assert.assertNotNull(replace1.getIndexIdByName("r3"));
@@ -690,16 +690,16 @@ public class AlterTest {
                 "PROPERTIES(\"replication_num\" = \"1\");";
 
         createTable(stmt);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
 
         String modifyBucketNumStmt = "ALTER TABLE test.bucket MODIFY DISTRIBUTION DISTRIBUTED BY HASH(k1) BUCKETS 1;";
         alterTable(modifyBucketNumStmt, false);
-        OlapTable bucket = (OlapTable) db.getTable("bucket");
+        OlapTable bucket = (OlapTable) db.getTableOrMetaException("bucket");
         Assert.assertEquals(1, bucket.getDefaultDistributionInfo().getBucketNum());
 
         modifyBucketNumStmt = "ALTER TABLE test.bucket MODIFY DISTRIBUTION DISTRIBUTED BY HASH(k1) BUCKETS 30;";
         alterTable(modifyBucketNumStmt, false);
-        bucket = (OlapTable) db.getTable("bucket");
+        bucket = (OlapTable) db.getTableOrMetaException("bucket");
         Assert.assertEquals(30, bucket.getDefaultDistributionInfo().getBucketNum());
 
     }
@@ -744,8 +744,8 @@ public class AlterTest {
         // external table support add column
         stmt = "alter table test.odbc_table add column k6 INT KEY after k1, add column k7 TINYINT KEY after k6";
         alterTable(stmt, false);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        Table odbc_table = db.getTable("odbc_table");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        Table odbc_table = db.getTableOrMetaException("odbc_table");
         Assert.assertEquals(odbc_table.getBaseSchema().size(), 7);
         Assert.assertEquals(odbc_table.getBaseSchema().get(1).getDataType(), PrimitiveType.INT);
         Assert.assertEquals(odbc_table.getBaseSchema().get(2).getDataType(), PrimitiveType.TINYINT);
@@ -753,21 +753,21 @@ public class AlterTest {
         // external table support drop column
         stmt = "alter table test.odbc_table drop column k7";
         alterTable(stmt, false);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        odbc_table = db.getTable("odbc_table");
+        db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        odbc_table = db.getTableOrMetaException("odbc_table");
         Assert.assertEquals(odbc_table.getBaseSchema().size(), 6);
 
         // external table support modify column
         stmt = "alter table test.odbc_table modify column k6 bigint after k5";
         alterTable(stmt, false);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        odbc_table = db.getTable("odbc_table");
+        db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        odbc_table = db.getTableOrMetaException("odbc_table");
         Assert.assertEquals(odbc_table.getBaseSchema().size(), 6);
         Assert.assertEquals(odbc_table.getBaseSchema().get(5).getDataType(), PrimitiveType.BIGINT);
 
         // external table support reorder column
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        odbc_table = db.getTable("odbc_table");
+        db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        odbc_table = db.getTableOrMetaException("odbc_table");
         Assert.assertTrue(odbc_table.getBaseSchema().stream().
                 map(column -> column.getName()).
                 reduce("", (totalName, columnName) -> totalName + columnName).equals("k1k2k3k4k5k6"));
@@ -797,10 +797,10 @@ public class AlterTest {
         // external table support rename operation
         stmt = "alter table test.odbc_table rename oracle_table";
         alterTable(stmt, false);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        odbc_table = db.getTable("oracle_table");
-        Assert.assertTrue(odbc_table != null);
-        odbc_table = db.getTable("odbc_table");
-        Assert.assertTrue(odbc_table == null);
+        db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        odbc_table = db.getTableNullable("oracle_table");
+        Assert.assertNotNull(odbc_table);
+        odbc_table = db.getTableNullable("odbc_table");
+        Assert.assertNull(odbc_table);
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
index 898deb3..2ccd1bc 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
@@ -80,9 +80,9 @@ public class BatchRollupJobTest {
         Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getRollupHandler().getAlterJobsV2();
         Assert.assertEquals(3, alterJobs.size());
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1");
         Assert.assertNotNull(db);
-        OlapTable tbl = (OlapTable) db.getTable("tbl1");
+        OlapTable tbl = (OlapTable) db.getTableNullable("tbl1");
         Assert.assertNotNull(tbl);
 
         int finishedNum = 0;
@@ -131,9 +131,9 @@ public class BatchRollupJobTest {
         Assert.assertEquals(3, alterJobs.size());
         List<Long> jobIds = Lists.newArrayList(alterJobs.keySet());
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1");
         Assert.assertNotNull(db);
-        OlapTable tbl = (OlapTable) db.getTable("tbl2");
+        OlapTable tbl = (OlapTable) db.getTableNullable("tbl2");
         Assert.assertNotNull(tbl);
 
         for (AlterJobV2 alterJobV2 : alterJobs.values()) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java
index 7293740..8a6f8f5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/RollupJobV2Test.java
@@ -156,8 +156,8 @@ public class RollupJobV2Test {
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(clause);
         alterClauses.add(clause2);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = materializedViewHandler.getAlterJobsV2();
 
@@ -176,8 +176,8 @@ public class RollupJobV2Test {
         MaterializedViewHandler materializedViewHandler = Catalog.getCurrentCatalog().getRollupHandler();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(clause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = materializedViewHandler.getAlterJobsV2();
         Assert.assertEquals(1, alterJobsV2.size());
@@ -195,8 +195,8 @@ public class RollupJobV2Test {
         // add a rollup job
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(clause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1);
         materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = materializedViewHandler.getAlterJobsV2();
@@ -299,8 +299,8 @@ public class RollupJobV2Test {
         // add a rollup job
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(clause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1);
         materializedViewHandler.process(alterClauses, db.getClusterName(), db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = materializedViewHandler.getAlterJobsV2();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java
index d299de2..864416d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeJobV2Test.java
@@ -135,8 +135,8 @@ public class SchemaChangeJobV2Test {
         SchemaChangeHandler schemaChangeHandler = Catalog.getCurrentCatalog().getSchemaChangeHandler();
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(addColumnClause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = schemaChangeHandler.getAlterJobsV2();
         Assert.assertEquals(1, alterJobsV2.size());
@@ -154,8 +154,8 @@ public class SchemaChangeJobV2Test {
         // add a schema change job
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(addColumnClause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1);
         schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = schemaChangeHandler.getAlterJobsV2();
@@ -230,8 +230,8 @@ public class SchemaChangeJobV2Test {
         // add a schema change job
         ArrayList<AlterClause> alterClauses = new ArrayList<>();
         alterClauses.add(addColumnClause);
-        Database db = masterCatalog.getDb(CatalogTestUtil.testDbId1);
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogTestUtil.testTableId1);
+        Database db = masterCatalog.getDbOrDdlException(CatalogTestUtil.testDbId1);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogTestUtil.testTableId1);
         Partition testPartition = olapTable.getPartition(CatalogTestUtil.testPartitionId1);
         schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable);
         Map<Long, AlterJobV2> alterJobsV2 = schemaChangeHandler.getAlterJobsV2();
@@ -317,7 +317,7 @@ public class SchemaChangeJobV2Test {
         properties.put(DynamicPartitionProperty.BUCKETS, "30");
         alterClauses.add(new ModifyTablePropertiesClause(properties));
         Database db = CatalogMocker.mockDb();
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogMocker.TEST_TBL2_ID);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogMocker.TEST_TBL2_ID);
         schemaChangeHandler.process(alterClauses, "default_cluster", db, olapTable);
         Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().isExist());
         Assert.assertTrue(olapTable.getTableProperty().getDynamicPartitionProperty().getEnable());
@@ -370,7 +370,7 @@ public class SchemaChangeJobV2Test {
         alterClauses.add(new ModifyTablePropertiesClause(properties));
 
         Database db = CatalogMocker.mockDb();
-        OlapTable olapTable = (OlapTable) db.getTable(CatalogMocker.TEST_TBL2_ID);
+        OlapTable olapTable = (OlapTable) db.getTableOrDdlException(CatalogMocker.TEST_TBL2_ID);
 
         expectedEx.expect(DdlException.class);
         expectedEx.expectMessage("errCode = 2, detailMessage = Table test_db.test_tbl2 is not a dynamic partition table. " +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AccessTestUtil.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AccessTestUtil.java
index d222a6a..6158705 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AccessTestUtil.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AccessTestUtil.java
@@ -115,19 +115,19 @@ public class AccessTestUtil {
                     minTimes = 0;
                     result = paloAuth;
 
-                    catalog.getDb(50000L);
+                    catalog.getDbNullable(50000L);
                     minTimes = 0;
                     result = db;
 
-                    catalog.getDb("testCluster:testDb");
+                    catalog.getDbNullable("testCluster:testDb");
                     minTimes = 0;
                     result = db;
 
-                    catalog.getDb("testCluster:emptyDb");
+                    catalog.getDbNullable("testCluster:emptyDb");
                     minTimes = 0;
                     result = null;
 
-                    catalog.getDb(anyString);
+                    catalog.getDbNullable(anyString);
                     minTimes = 0;
                     result = new Database();
 
@@ -234,11 +234,11 @@ public class AccessTestUtil {
 
         new Expectations(db) {
             {
-                db.getTable("testTable");
+                db.getTableNullable("testTable");
                 minTimes = 0;
                 result = olapTable;
 
-                db.getTable("emptyTable");
+                db.getTableNullable("emptyTable");
                 minTimes = 0;
                 result = null;
 
@@ -281,15 +281,15 @@ public class AccessTestUtil {
                     minTimes = 0;
                     result = new DdlException("failed");
 
-                    catalog.getDb("testCluster:testDb");
+                    catalog.getDbNullable("testCluster:testDb");
                     minTimes = 0;
                     result = db;
 
-                    catalog.getDb("testCluster:emptyDb");
+                    catalog.getDbNullable("testCluster:emptyDb");
                     minTimes = 0;
                     result = null;
 
-                    catalog.getDb(anyString);
+                    catalog.getDbNullable(anyString);
                     minTimes = 0;
                     result = new Database();
 
@@ -301,7 +301,7 @@ public class AccessTestUtil {
                     minTimes = 0;
                     result = Lists.newArrayList("testCluster:testDb");
 
-                    catalog.getDb("emptyCluster");
+                    catalog.getDbNullable("emptyCluster");
                     minTimes = 0;
                     result = null;
                 }
@@ -388,7 +388,7 @@ public class AccessTestUtil {
         return analyzer;
     }
 
-    public static Analyzer fetchTableAnalyzer() {
+    public static Analyzer fetchTableAnalyzer() throws AnalysisException {
         Column column1 = new Column("k1", PrimitiveType.VARCHAR);
         Column column2 = new Column("k2", PrimitiveType.VARCHAR);
         Column column3 = new Column("k3", PrimitiveType.VARCHAR);
@@ -449,11 +449,11 @@ public class AccessTestUtil {
 
         new Expectations(db) {
             {
-                db.getTable("t");
+                db.getTableNullable("t");
                 minTimes = 0;
                 result = table;
 
-                db.getTable("emptyTable");
+                db.getTableNullable("emptyTable");
                 minTimes = 0;
                 result = null;
 
@@ -484,7 +484,7 @@ public class AccessTestUtil {
                 minTimes = 0;
                 result = "testDb";
 
-                analyzer.getTable((TableName) any);
+                analyzer.getTableOrAnalysisException((TableName) any);
                 minTimes = 0;
                 result = table;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
index 816600d..c67cf8c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
@@ -101,8 +101,8 @@ public class AdminShowReplicaTest {
         Assert.assertEquals(4, resultSet.getResultRows().get(0).size());
 
         // update tablets' data size and row count
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable olapTable = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
+        OlapTable olapTable = db.getOlapTableOrAnalysisException("tbl1");
         for (Partition partition : olapTable.getPartitions()) {
             for (MaterializedIndex mIndex : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE)) {
                 for (Tablet tablet : mIndex.getTablets()) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterViewStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterViewStmtTest.java
index df0f13f..6466120 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterViewStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AlterViewStmtTest.java
@@ -25,7 +25,6 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.SinglePartitionInfo;
 import org.apache.doris.catalog.View;
-import org.apache.doris.common.UserException;
 import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.common.util.SqlParserUtils;
 import org.apache.doris.mysql.privilege.PaloAuth;
@@ -122,11 +121,19 @@ public class AlterViewStmtTest {
                 return auth;
             }
             @Mock
-            Database getDb(long dbId) {
+            Database getDbOrDdlException(long dbId) {
                 return db;
             }
             @Mock
-            Database getDb(String dbName) {
+            Database getDbOrDdlException(String dbName) {
+                return db;
+            }
+            @Mock
+            Database getDbOrAnalysisException(long dbId) {
+                return db;
+            }
+            @Mock
+            Database getDbOrAnalysisException(String dbName) {
                 return db;
             }
         };
@@ -140,52 +147,36 @@ public class AlterViewStmtTest {
     }
 
     @Test
-    public void testNormal() {
+    public void testNormal() throws Exception {
         String originStmt = "select col1 as c1, sum(col2) as c2 from testDb.testTbl group by col1";
         View view = new View(30000L, "testView", null);
         view.setInlineViewDefWithSqlMode("select col1 as c1, sum(col2) as c2 from testDb.testTbl group by col1", 0L);
-        try {
-            view.init();
-        } catch (UserException e) {
-            Assert.fail();
-        }
+        view.init();
 
-        Database db = analyzer.getCatalog().getDb("testDb");
+        Database db = analyzer.getCatalog().getDbOrAnalysisException("testDb");
         db.createTable(view);
 
         Assert.assertEquals(originStmt, view.getInlineViewDef());
 
         String alterStmt = "with testTbl_cte (w1, w2) as (select col1, col2 from testDb.testTbl) select w1 as c1, sum(w2) as c2 from testTbl_cte where w1 > 10 group by w1 order by w1";
         SqlParser parser = new SqlParser(new SqlScanner(new StringReader(alterStmt)));
-        QueryStmt alterQueryStmt = null;
-        try {
-            alterQueryStmt = (QueryStmt) SqlParserUtils.getFirstStmt(parser);
-        } catch (Error e) {
-            Assert.fail(e.getMessage());
-        } catch (Exception e) {
-            Assert.fail(e.getMessage());
-        }
+        QueryStmt alterQueryStmt = (QueryStmt) SqlParserUtils.getFirstStmt(parser);
 
         ColWithComment col1 = new ColWithComment("h1", null);
         ColWithComment col2 = new ColWithComment("h2", null);
 
         AlterViewStmt alterViewStmt = new AlterViewStmt(new TableName("testDb", "testView"), Lists.newArrayList(col1, col2), alterQueryStmt);
-        try {
-            alterViewStmt.analyze(analyzer);
-            Catalog catalog1 = analyzer.getCatalog();
-            if (catalog1 == null) {
-                System.out.println("cmy get null");
-                return;
-            }
-            catalog1.alterView(alterViewStmt);
-        } catch (UserException e) {
-            Assert.fail();
+        alterViewStmt.analyze(analyzer);
+        Catalog catalog1 = analyzer.getCatalog();
+        if (catalog1 == null) {
+            System.out.println("cmy get null");
+            return;
         }
+        catalog1.alterView(alterViewStmt);
 
-        View newView = (View) db.getTable("testView");
-
-        Assert.assertEquals("WITH testTbl_cte(w1, w2) AS (SELECT `col1` AS `col1`, `col2` AS `col2` FROM `testCluster:testDb`.`testTbl`)"+
-                                     " SELECT `w1` AS `h1`, sum(`w2`) AS `h2` FROM `testTbl_cte` WHERE `w1` > 10 GROUP BY `w1` ORDER BY `w1`",
+        View newView = (View) db.getTableOrAnalysisException("testView");
+        Assert.assertEquals("WITH testTbl_cte(w1, w2) AS (SELECT `col1` AS `col1`, `col2` AS `col2` FROM `testCluster:testDb`.`testTbl`)" +
+                        " SELECT `w1` AS `h1`, sum(`w2`) AS `h2` FROM `testTbl_cte` WHERE `w1` > 10 GROUP BY `w1` ORDER BY `w1`",
                 newView.getInlineViewDef());
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDataSyncJobStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDataSyncJobStmtTest.java
index fa029ac..de3ef16 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDataSyncJobStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateDataSyncJobStmtTest.java
@@ -68,7 +68,7 @@ public class CreateDataSyncJobStmtTest {
         properties = Maps.newHashMap();
         new Expectations() {
             {
-                catalog.getDb("testCluster:testDb");
+                catalog.getDbNullable("testCluster:testDb");
                 minTimes = 0;
                 result = database;
 
@@ -84,7 +84,7 @@ public class CreateDataSyncJobStmtTest {
                 minTimes = 0;
                 result = true;
 
-                database.getTable("testTbl");
+                database.getTableNullable("testTbl");
                 minTimes = 0;
                 result = table;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java
index 1607733..bce2e07 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/CreateRoutineLoadStmtTest.java
@@ -70,11 +70,11 @@ public class CreateRoutineLoadStmtTest {
         };
         new Expectations() {
             {
-                catalog.getDb(anyString);
+                catalog.getDbNullable(anyString);
                 minTimes = 0;
                 result = database;
 
-                database.getTable(anyString);
+                database.getTableNullable(anyString);
                 minTimes = 0;
                 result = table;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java
index 85395eb..dee1b0f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/DataDescriptionTest.java
@@ -80,11 +80,11 @@ public class DataDescriptionTest {
                 minTimes = 0;
                 result = catalog;
 
-                catalog.getDb(anyString);
+                catalog.getDbNullable(anyString);
                 minTimes = 0;
                 result = db;
 
-                db.getTable(anyString);
+                db.getTableNullable(anyString);
                 minTimes = 0;
                 result = tbl;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/GroupByClauseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/GroupByClauseTest.java
index aa9f505..e105863 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/GroupByClauseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/GroupByClauseTest.java
@@ -39,7 +39,7 @@ public class GroupByClauseTest {
     private Analyzer analyzer;
 
     @Before
-    public void setUp() {
+    public void setUp() throws AnalysisException {
         Analyzer analyzerBase = AccessTestUtil.fetchTableAnalyzer();
         analyzer = new Analyzer(analyzerBase.getCatalog(), analyzerBase.getContext());
         try {
@@ -47,7 +47,7 @@ public class GroupByClauseTest {
             f.setAccessible(true);
             Multimap<String, TupleDescriptor> tupleByAlias = ArrayListMultimap.create();
             TupleDescriptor td = new TupleDescriptor(new TupleId(0));
-            td.setTable(analyzerBase.getTable(new TableName("testdb", "t")));
+            td.setTable(analyzerBase.getTableOrAnalysisException(new TableName("testdb", "t")));
             tupleByAlias.put("testdb.t", td);
             f.set(analyzer, tupleByAlias);
         } catch (NoSuchFieldException e) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java
index a9209e0..325eaf9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowDataStmtTest.java
@@ -53,7 +53,7 @@ public class ShowDataStmtTest {
     private Database db;
 
     @Before
-    public void setUp() throws AnalysisException {
+    public void setUp() throws UserException {
         auth = new PaloAuth();
 
         
@@ -94,7 +94,7 @@ public class ShowDataStmtTest {
                 minTimes = 0;
                 result = auth;
 
-                catalog.getDb(anyString);
+                catalog.getDbOrAnalysisException(anyString);
                 minTimes = 0;
                 result = db;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
index ce464ce..bee5637 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
@@ -81,7 +81,7 @@ public class TableNameComparedLowercaseTest {
 
     @Test
     public void testTableNameLowerCase() {
-        Set<String> tableNames = Catalog.getCurrentCatalog().getDb("default_cluster:db1").getTableNamesWithLock();
+        Set<String> tableNames = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1").getTableNamesWithLock();
         Assert.assertEquals(2, tableNames.size());
         Assert.assertTrue(tableNames.contains("TABLE1"));
         Assert.assertTrue(tableNames.contains("TABLE2"));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
index 400ea65..3bf64fa 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
@@ -80,7 +80,7 @@ public class TableNameStoredLowercaseTest {
 
     @Test
     public void testTableNameLowerCase() {
-        Set<String> tableNames = Catalog.getCurrentCatalog().getDb("default_cluster:db1").getTableNamesWithLock();
+        Set<String> tableNames = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1").getTableNamesWithLock();
         Assert.assertEquals(2, tableNames.size());
         Assert.assertTrue(tableNames.contains("table1"));
         Assert.assertTrue(tableNames.contains("table2"));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/VirtualSlotRefTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/VirtualSlotRefTest.java
index e37a129..c505316 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/VirtualSlotRefTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/VirtualSlotRefTest.java
@@ -22,6 +22,7 @@ import org.apache.doris.catalog.Type;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 
+import org.apache.doris.common.AnalysisException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -47,7 +48,7 @@ public class VirtualSlotRefTest {
     DataInputStream dis;
 
     @Before
-    public void setUp() throws IOException {
+    public void setUp() throws IOException, AnalysisException {
         Analyzer analyzerBase = AccessTestUtil.fetchTableAnalyzer();
         analyzer = new Analyzer(analyzerBase.getCatalog(), analyzerBase.getContext());
         String[] cols = {"k1", "k2", "k3"};
@@ -61,12 +62,10 @@ public class VirtualSlotRefTest {
             f.setAccessible(true);
             Multimap<String, TupleDescriptor> tupleByAlias = ArrayListMultimap.create();
             TupleDescriptor td = new TupleDescriptor(new TupleId(0));
-            td.setTable(analyzerBase.getTable(new TableName("testdb", "t")));
+            td.setTable(analyzerBase.getTableOrAnalysisException(new TableName("testdb", "t")));
             tupleByAlias.put("testdb.t", td);
             f.set(analyzer, tupleByAlias);
-        } catch (NoSuchFieldException e) {
-            e.printStackTrace();
-        } catch (IllegalAccessException e) {
+        } catch (NoSuchFieldException | IllegalAccessException e) {
             e.printStackTrace();
         }
         virtualTuple = analyzer.getDescTbl().createTupleDescriptor("VIRTUAL_TUPLE");
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
index ffc5196..84edd92 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupHandlerTest.java
@@ -38,9 +38,7 @@ import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletInvertedIndex;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
-import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.persist.EditLog;
 import org.apache.doris.task.DirMoveTask;
@@ -100,7 +98,7 @@ public class BackupHandlerTest {
     private TabletInvertedIndex invertedIndex = new TabletInvertedIndex();
 
     @Before
-    public void setUp() {
+    public void setUp() throws Exception {
         Config.tmp_dir = tmpPath;
         rootDir = new File(Config.tmp_dir);
         rootDir.mkdirs();
@@ -133,16 +131,11 @@ public class BackupHandlerTest {
             }
         };
 
-        try {
-            db = CatalogMocker.mockDb();
-        } catch (AnalysisException e) {
-            e.printStackTrace();
-            Assert.fail();
-        }
+        db = CatalogMocker.mockDb();
 
         new Expectations() {
             {
-                catalog.getDb(anyString);
+                catalog.getDbOrDdlException(anyString);
                 minTimes = 0;
                 result = db;
             }
@@ -172,7 +165,7 @@ public class BackupHandlerTest {
     }
 
     @Test
-    public void testCreateAndDropRepository() {
+    public void testCreateAndDropRepository() throws Exception {
         new Expectations() {
             {
                 editLog.logCreateRepository((Repository) any);
@@ -206,8 +199,8 @@ public class BackupHandlerTest {
             }
 
             @Mock
-            public Status getSnapshotInfoFile(String label, String backupTimestamp, List<BackupJobInfo> infos) {
-                OlapTable tbl = (OlapTable) db.getTable(CatalogMocker.TEST_TBL_NAME);
+            public Status getSnapshotInfoFile(String label, String backupTimestamp, List<BackupJobInfo> infos) throws Exception {
+                OlapTable tbl = (OlapTable) db.getTableOrMetaException(CatalogMocker.TEST_TBL_NAME);
                 List<Table> tbls = Lists.newArrayList();
                 tbls.add(tbl);
                 List<Resource> resources = Lists.newArrayList();
@@ -218,7 +211,7 @@ public class BackupHandlerTest {
                         for (Tablet tablet : idx.getTablets()) {
                             List<String> files = Lists.newArrayList();
                             SnapshotInfo sinfo = new SnapshotInfo(db.getId(), tbl.getId(), part.getId(), idx.getId(),
-                                                                  tablet.getId(), -1, 0, "./path", files);
+                                    tablet.getId(), -1, 0, "./path", files);
                             snapshotInfos.put(tablet.getId(), sinfo);
                         }
                     }
@@ -244,14 +237,9 @@ public class BackupHandlerTest {
         // add repo
         handler = new BackupHandler(catalog);
         StorageBackend storageBackend = new StorageBackend("broker", "bos://location",
-                StorageBackend.StorageType.BROKER ,Maps.newHashMap());
+                StorageBackend.StorageType.BROKER, Maps.newHashMap());
         CreateRepositoryStmt stmt = new CreateRepositoryStmt(false, "repo", storageBackend);
-        try {
-            handler.createRepository(stmt);
-        } catch (DdlException e) {
-            e.printStackTrace();
-            Assert.fail();
-        }
+        handler.createRepository(stmt);
 
         // process backup
         List<TableRef> tblRefs = Lists.newArrayList();
@@ -259,17 +247,12 @@ public class BackupHandlerTest {
         AbstractBackupTableRefClause tableRefClause = new AbstractBackupTableRefClause(false, tblRefs);
         BackupStmt backupStmt = new BackupStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "label1"), "repo",
                 tableRefClause, null);
-        try {
-            handler.process(backupStmt);
-        } catch (DdlException e1) {
-            e1.printStackTrace();
-            Assert.fail();
-        }
+        handler.process(backupStmt);
 
         // handleFinishedSnapshotTask
         BackupJob backupJob = (BackupJob) handler.getJob(CatalogMocker.TEST_DB_ID);
         SnapshotTask snapshotTask = new SnapshotTask(null, 0, 0, backupJob.getJobId(), CatalogMocker.TEST_DB_ID,
-                                                     0, 0, 0, 0, 0, 0, 0, 1, false);
+                0, 0, 0, 0, 0, 0, 0, 1, false);
         TFinishTaskRequest request = new TFinishTaskRequest();
         List<String> snapshotFiles = Lists.newArrayList();
         request.setSnapshotFiles(snapshotFiles);
@@ -297,20 +280,12 @@ public class BackupHandlerTest {
             DataInputStream in = new DataInputStream(new FileInputStream(tmpFile));
             BackupHandler.read(in);
             in.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-            Assert.fail();
         } finally {
             tmpFile.delete();
         }
 
         // cancel backup
-        try {
-            handler.cancel(new CancelBackupStmt(CatalogMocker.TEST_DB_NAME, false));
-        } catch (DdlException e1) {
-            e1.printStackTrace();
-            Assert.fail();
-        }
+        handler.cancel(new CancelBackupStmt(CatalogMocker.TEST_DB_NAME, false));
 
         // process restore
         List<TableRef> tblRefs2 = Lists.newArrayList();
@@ -320,19 +295,9 @@ public class BackupHandlerTest {
         AbstractBackupTableRefClause abstractBackupTableRefClause = new AbstractBackupTableRefClause(false, tblRefs2);
         RestoreStmt restoreStmt = new RestoreStmt(new LabelName(CatalogMocker.TEST_DB_NAME, "ss2"), "repo",
                 abstractBackupTableRefClause, properties);
-        try {
-            restoreStmt.analyzeProperties();
-        } catch (AnalysisException e2) {
-            e2.printStackTrace();
-            Assert.fail();
-        }
 
-        try {
-            handler.process(restoreStmt);
-        } catch (DdlException e1) {
-            e1.printStackTrace();
-            Assert.fail();
-        }
+        restoreStmt.analyzeProperties();
+        handler.process(restoreStmt);
 
         // handleFinishedSnapshotTask
         RestoreJob restoreJob = (RestoreJob) handler.getJob(CatalogMocker.TEST_DB_ID);
@@ -369,27 +334,14 @@ public class BackupHandlerTest {
             DataInputStream in = new DataInputStream(new FileInputStream(tmpFile));
             BackupHandler.read(in);
             in.close();
-        } catch (IOException e) {
-            e.printStackTrace();
-            Assert.fail();
         } finally {
             tmpFile.delete();
         }
 
         // cancel restore
-        try {
-            handler.cancel(new CancelBackupStmt(CatalogMocker.TEST_DB_NAME, true));
-        } catch (DdlException e1) {
-            e1.printStackTrace();
-            Assert.fail();
-        }
+        handler.cancel(new CancelBackupStmt(CatalogMocker.TEST_DB_NAME, true));
 
         // drop repo
-        try {
-            handler.dropRepository(new DropRepositoryStmt("repo"));
-        } catch (DdlException e) {
-            e.printStackTrace();
-            Assert.fail();
-        }
+        handler.dropRepository(new DropRepositoryStmt("repo"));
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
index 0c10496..d6a76a7 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java
@@ -149,7 +149,7 @@ public class BackupJobTest {
 
         new Expectations(catalog) {
             {
-                catalog.getDb(anyLong);
+                catalog.getDbNullable(anyLong);
                 minTimes = 0;
                 result = db;
 
@@ -219,7 +219,7 @@ public class BackupJobTest {
         List<String> partNames = Lists.newArrayList(backupTbl.getPartitionNames());
         Assert.assertNotNull(backupTbl);
         Assert.assertEquals(backupTbl.getSignature(BackupHandler.SIGNATURE_VERSION, partNames),
-                            ((OlapTable) db.getTable(tblId)).getSignature(BackupHandler.SIGNATURE_VERSION, partNames));
+                            ((OlapTable) db.getTableNullable(tblId)).getSignature(BackupHandler.SIGNATURE_VERSION, partNames));
         Assert.assertEquals(1, AgentTaskQueue.getTaskNum());
         AgentTask task = AgentTaskQueue.getTask(backendId, TTaskType.MAKE_SNAPSHOT, tabletId);
         Assert.assertTrue(task instanceof SnapshotTask);
@@ -308,7 +308,7 @@ public class BackupJobTest {
             Assert.assertNotNull(olapTable);
             Assert.assertNotNull(restoreMetaInfo.getTable(UnitTestUtil.TABLE_NAME));
             List<String> names = Lists.newArrayList(olapTable.getPartitionNames());
-            Assert.assertEquals(((OlapTable) db.getTable(tblId)).getSignature(BackupHandler.SIGNATURE_VERSION, names),
+            Assert.assertEquals(((OlapTable) db.getTableNullable(tblId)).getSignature(BackupHandler.SIGNATURE_VERSION, names),
                                 olapTable.getSignature(BackupHandler.SIGNATURE_VERSION, names));
 
             restoreJobInfo = BackupJobInfo.fromFile(job.getLocalJobInfoFilePath());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java b/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
index da9166c..230bdab 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
@@ -45,8 +45,8 @@ import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.SinglePartitionInfo;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletMeta;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.load.Load;
@@ -228,7 +228,7 @@ public class CatalogMocker {
         return clusterInfo;
     }
 
-    public static Database mockDb() throws AnalysisException {
+    public static Database mockDb() throws UserException {
         // mock all meta obj
         Database db = new Database(TEST_DB_ID, TEST_DB_NAME);
 
@@ -405,19 +405,19 @@ public class CatalogMocker {
                     minTimes = 0;
                     result = paloAuth;
 
-                    catalog.getDb(TEST_DB_NAME);
+                    catalog.getDbNullable(TEST_DB_NAME);
                     minTimes = 0;
                     result = db;
 
-                    catalog.getDb(WRONG_DB);
+                    catalog.getDbNullable(WRONG_DB);
                     minTimes = 0;
                     result = null;
 
-                    catalog.getDb(TEST_DB_ID);
+                    catalog.getDbNullable(TEST_DB_ID);
                     minTimes = 0;
                     result = db;
 
-                    catalog.getDb(anyString);
+                    catalog.getDbNullable(anyString);
                     minTimes = 0;
                     result = new Database();
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
index bf7fdf6..dd70058 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
@@ -125,7 +125,7 @@ public class RestoreJobTest {
     private BackupMeta backupMeta;
 
     @Before
-    public void setUp() throws AnalysisException {
+    public void setUp() throws Exception {
         db = CatalogMocker.mockDb();
         backupHandler = new MockBackupHandler(catalog);
         repoMgr = new MockRepositoryMgr();
@@ -134,7 +134,7 @@ public class RestoreJobTest {
 
         new Expectations() {
             {
-                catalog.getDb(anyLong);
+                catalog.getDbNullable(anyLong);
                 minTimes = 0;
                 result = db;
         
@@ -218,7 +218,7 @@ public class RestoreJobTest {
         jobInfo.name = label;
         jobInfo.success = true;
         
-        expectedRestoreTbl = (OlapTable) db.getTable(CatalogMocker.TEST_TBL2_ID);
+        expectedRestoreTbl = (OlapTable) db.getTableNullable(CatalogMocker.TEST_TBL2_ID);
         BackupOlapTableInfo tblInfo = new BackupOlapTableInfo();
         tblInfo.id = CatalogMocker.TEST_TBL2_ID;
         jobInfo.backupOlapTableObjects.put(CatalogMocker.TEST_TBL2_NAME, tblInfo);
@@ -363,7 +363,7 @@ public class RestoreJobTest {
     }
 
     @Test
-    public void testSignature() {
+    public void testSignature() throws AnalysisException {
         Adler32 sig1 = new Adler32();
         sig1.update("name1".getBytes());
         sig1.update("name2".getBytes());
@@ -374,7 +374,7 @@ public class RestoreJobTest {
         sig2.update("name1".getBytes());
         System.out.println("sig2: " + Math.abs((int) sig2.getValue()));
 
-        OlapTable tbl = (OlapTable) db.getTable(CatalogMocker.TEST_TBL_NAME);
+        OlapTable tbl = db.getOlapTableOrAnalysisException(CatalogMocker.TEST_TBL_NAME);
         List<String> partNames = Lists.newArrayList(tbl.getPartitionNames());
         System.out.println(partNames);
         System.out.println("tbl signature: " + tbl.getSignature(BackupHandler.SIGNATURE_VERSION, partNames));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
index d4dff17..8f8b603 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
@@ -84,9 +84,9 @@ public class AdminStmtTest {
 
     @Test
     public void testAdminSetReplicaStatus() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test");
         Assert.assertNotNull(db);
-        OlapTable tbl = (OlapTable) db.getTable("tbl1");
+        OlapTable tbl = (OlapTable) db.getTableNullable("tbl1");
         Assert.assertNotNull(tbl);
         // tablet id, backend id
         List<Pair<Long, Long>> tabletToBackendList = Lists.newArrayList();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
index a85a5d2..1d54545 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
@@ -110,13 +110,13 @@ public class CatalogOperationTest {
         String renameTblStmt = "alter table test.renameTest rename newNewTest";
         AlterTableStmt alterTableStmt = (AlterTableStmt)UtFrameUtils.parseAndAnalyzeStmt(renameTblStmt, connectContext);
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test");
         Assert.assertNotNull(db);
-        Assert.assertNotNull(db.getTable("renameTest"));
+        Assert.assertNotNull(db.getTableNullable("renameTest"));
 
         Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        Assert.assertNull(db.getTable("renameTest"));
-        Assert.assertNotNull(db.getTable("newNewTest"));
+        Assert.assertNull(db.getTableNullable("renameTest"));
+        Assert.assertNotNull(db.getTableNullable("newNewTest"));
 
         // add a rollup and test rename to a rollup name(expect throw exception)
         String alterStmtStr = "alter table test.newNewTest add rollup r1(k1)";
@@ -145,16 +145,16 @@ public class CatalogOperationTest {
         renameTblStmt = "alter table test.newNewTest rename goodName";
         alterTableStmt = (AlterTableStmt)UtFrameUtils.parseAndAnalyzeStmt(renameTblStmt, connectContext);
         Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        Assert.assertNull(db.getTable("newNewTest"));
-        Assert.assertNotNull(db.getTable("goodName"));
+        Assert.assertNull(db.getTableNullable("newNewTest"));
+        Assert.assertNotNull(db.getTableNullable("goodName"));
 
         // rename external table
         renameTblStmt = "alter table test.mysqlRenameTest rename newMysqlRenameTest";
         alterTableStmt = (AlterTableStmt)UtFrameUtils.parseAndAnalyzeStmt(renameTblStmt, connectContext);
-        Assert.assertNotNull(db.getTable("mysqlRenameTest"));
+        Assert.assertNotNull(db.getTableNullable("mysqlRenameTest"));
 
         Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
-        Assert.assertNull(db.getTable("mysqlRenameTest"));
-        Assert.assertNotNull(db.getTable("newMysqlRenameTest"));
+        Assert.assertNull(db.getTableNullable("mysqlRenameTest"));
+        Assert.assertNotNull(db.getTableNullable("newMysqlRenameTest"));
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
index 0ec5a1b..8e2eb77 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.MaterializedIndex.IndexState;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.persist.EditLog;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
@@ -106,58 +107,59 @@ public class CatalogTestUtil {
     }
 
     public static boolean compareCatalog(Catalog masterCatalog, Catalog slaveCatalog) {
-        Database masterDb = masterCatalog.getDb(testDb1);
-        Database slaveDb = slaveCatalog.getDb(testDb1);
-        List<Table> tables = masterDb.getTables();
-        for (Table table : tables) {
-            Table slaveTable = slaveDb.getTable(table.getId());
-            if (slaveTable == null) {
-                return false;
-            }
-            Partition masterPartition = table.getPartition(testPartition1);
-            Partition slavePartition = slaveTable.getPartition(testPartition1);
-            if (masterPartition == null && slavePartition == null) {
-                return true;
-            }
-            if (masterPartition.getId() != slavePartition.getId()) {
-                return false;
-            }
-            if (masterPartition.getVisibleVersion() != slavePartition.getVisibleVersion()
-                    || masterPartition.getVisibleVersionHash() != slavePartition.getVisibleVersionHash()
-                    || masterPartition.getNextVersion() != slavePartition.getNextVersion()
-                    || masterPartition.getCommittedVersionHash() != slavePartition.getCommittedVersionHash()) {
-                return false;
-            }
-            List<MaterializedIndex> allMaterializedIndices = masterPartition.getMaterializedIndices(IndexExtState.ALL);
-            for (MaterializedIndex masterIndex : allMaterializedIndices) {
-                MaterializedIndex slaveIndex = slavePartition.getIndex(masterIndex.getId());
-                if (slaveIndex == null) {
+        try {
+            Database masterDb = masterCatalog.getDbOrMetaException(testDb1);
+            Database slaveDb = slaveCatalog.getDbOrMetaException(testDb1);
+            List<Table> tables = masterDb.getTables();
+            for (Table table : tables) {
+                Table slaveTable = slaveDb.getTableOrMetaException(table.getId());
+                Partition masterPartition = table.getPartition(testPartition1);
+                Partition slavePartition = slaveTable.getPartition(testPartition1);
+                if (masterPartition == null && slavePartition == null) {
+                    return true;
+                }
+                if (masterPartition.getId() != slavePartition.getId()) {
+                    return false;
+                }
+                if (masterPartition.getVisibleVersion() != slavePartition.getVisibleVersion()
+                        || masterPartition.getVisibleVersionHash() != slavePartition.getVisibleVersionHash()
+                        || masterPartition.getNextVersion() != slavePartition.getNextVersion()
+                        || masterPartition.getCommittedVersionHash() != slavePartition.getCommittedVersionHash()) {
                     return false;
                 }
-                List<Tablet> allTablets = masterIndex.getTablets();
-                for (Tablet masterTablet : allTablets) {
-                    Tablet slaveTablet = slaveIndex.getTablet(masterTablet.getId());
-                    if (slaveTablet == null) {
+                List<MaterializedIndex> allMaterializedIndices = masterPartition.getMaterializedIndices(IndexExtState.ALL);
+                for (MaterializedIndex masterIndex : allMaterializedIndices) {
+                    MaterializedIndex slaveIndex = slavePartition.getIndex(masterIndex.getId());
+                    if (slaveIndex == null) {
                         return false;
                     }
-                    List<Replica> allReplicas = masterTablet.getReplicas();
-                    for (Replica masterReplica : allReplicas) {
-                        Replica slaveReplica = slaveTablet.getReplicaById(masterReplica.getId());
-                        if (slaveReplica.getBackendId() != masterReplica.getBackendId()
-                                || slaveReplica.getVersion() != masterReplica.getVersion()
-                                || slaveReplica.getVersionHash() != masterReplica.getVersionHash()
-                                || slaveReplica.getLastFailedVersion() != masterReplica.getLastFailedVersion()
-                                || slaveReplica.getLastFailedVersionHash() != masterReplica.getLastFailedVersionHash()
-                                || slaveReplica.getLastSuccessVersion() != slaveReplica.getLastSuccessVersion()
-                                || slaveReplica.getLastSuccessVersionHash() != slaveReplica
-                                        .getLastSuccessVersionHash()) {
+                    List<Tablet> allTablets = masterIndex.getTablets();
+                    for (Tablet masterTablet : allTablets) {
+                        Tablet slaveTablet = slaveIndex.getTablet(masterTablet.getId());
+                        if (slaveTablet == null) {
                             return false;
                         }
+                        List<Replica> allReplicas = masterTablet.getReplicas();
+                        for (Replica masterReplica : allReplicas) {
+                            Replica slaveReplica = slaveTablet.getReplicaById(masterReplica.getId());
+                            if (slaveReplica.getBackendId() != masterReplica.getBackendId()
+                                    || slaveReplica.getVersion() != masterReplica.getVersion()
+                                    || slaveReplica.getVersionHash() != masterReplica.getVersionHash()
+                                    || slaveReplica.getLastFailedVersion() != masterReplica.getLastFailedVersion()
+                                    || slaveReplica.getLastFailedVersionHash() != masterReplica.getLastFailedVersionHash()
+                                    || slaveReplica.getLastSuccessVersion() != slaveReplica.getLastSuccessVersion()
+                                    || slaveReplica.getLastSuccessVersionHash() != slaveReplica
+                                    .getLastSuccessVersionHash()) {
+                                return false;
+                            }
+                        }
                     }
                 }
             }
+            return true;
+        } catch (MetaNotFoundException e) {
+            return false;
         }
-        return true;
     }
 
     public static Database createSimpleDb(long dbId, long tableId, long partitionId, long indexId, long tabletId,
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
index 3accf55..a4afb90 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
@@ -104,8 +104,8 @@ public class ColocateTableTest {
                 ");");
 
         ColocateTableIndex index = Catalog.getCurrentColocateIndex();
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        long tableId = db.getTable(tableName1).getId();
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        long tableId = db.getTableOrMetaException(tableName1).getId();
 
         Assert.assertEquals(1, Deencapsulation.<Multimap<GroupId, Long>>getField(index, "group2Tables").size());
         Assert.assertEquals(1, index.getAllGroupIds().size());
@@ -159,9 +159,9 @@ public class ColocateTableTest {
                 ");");
 
         ColocateTableIndex index = Catalog.getCurrentColocateIndex();
-        Database db = Catalog.getCurrentCatalog().getDb(fullDbName);
-        long firstTblId = db.getTable(tableName1).getId();
-        long secondTblId = db.getTable(tableName2).getId();
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
+        long firstTblId = db.getTableOrMetaException(tableName1).getId();
+        long secondTblId = db.getTableOrMetaException(tableName2).getId();
 
         Assert.assertEquals(2, Deencapsulation.<Multimap<GroupId, Long>>getField(index, "group2Tables").size());
         Assert.assertEquals(1, index.getAllGroupIds().size());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
index cbd829e..0bfe7fd 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
@@ -64,7 +64,7 @@ public class CreateEncryptKeyTest {
         Catalog.getCurrentCatalog().createDb(createDbStmt);
         System.out.println(Catalog.getCurrentCatalog().getDbNames());
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1");
         Assert.assertNotNull(db);
 
         String createFuncStr = "create encryptkey db1.my_key as \"beijing\";";
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
index c744ef0..0e4f90a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
@@ -71,7 +71,7 @@ public class CreateFunctionTest {
         Catalog.getCurrentCatalog().createDb(createDbStmt);
         System.out.println(Catalog.getCurrentCatalog().getDbNames());
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1");
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:db1");
         Assert.assertNotNull(db);
 
         String createFuncStr = "create function db1.my_add(VARCHAR(1024)) RETURNS BOOLEAN properties\n" +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
index 858eb76..127dd65 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
@@ -110,10 +110,10 @@ public class CreateTableLikeTest {
                                                  String newTblName, String existedTblName, int rollupSize) throws Exception {
         createTable(createTableSql);
         createTableLike(createTableLikeSql);
-        Database newDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + newDbName);
-        Database existedDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + existedDbName);
-        OlapTable newTbl = (OlapTable) newDb.getTable(newTblName);
-        OlapTable existedTbl = (OlapTable) existedDb.getTable(existedTblName);
+        Database newDb = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:" + newDbName);
+        Database existedDb = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:" + existedDbName);
+        OlapTable newTbl = (OlapTable) newDb.getTableOrDdlException(newTblName);
+        OlapTable existedTbl = (OlapTable) existedDb.getTableOrDdlException(existedTblName);
         checkTableEqual(newTbl, existedTbl, rollupSize);
     }
 
@@ -123,10 +123,10 @@ public class CreateTableLikeTest {
 
         createTable(createTableSql);
         createTableLike(createTableLikeSql);
-        Database newDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + newDbName);
-        Database existedDb = Catalog.getCurrentCatalog().getDb("default_cluster:" + existedDbName);
-        MysqlTable newTbl = (MysqlTable) newDb.getTable(newTblName);
-        MysqlTable existedTbl = (MysqlTable) existedDb.getTable(existedTblName);
+        Database newDb = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:" + newDbName);
+        Database existedDb = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:" + existedDbName);
+        MysqlTable newTbl = (MysqlTable) newDb.getTableOrDdlException(newTblName);
+        MysqlTable existedTbl = (MysqlTable) existedDb.getTableOrDdlException(existedTblName);
         checkTableEqual(newTbl, existedTbl, 0);
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
index 6af49d2..71f3b6f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
@@ -198,18 +198,18 @@ public class CreateTableTest {
                         + "distributed by hash(k2) buckets 1\n"
                         + "properties('replication_num' = '1');"));
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable tbl6 = (OlapTable) db.getTable("tbl6");
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:test");
+        OlapTable tbl6 = (OlapTable) db.getTableOrDdlException("tbl6");
         Assert.assertTrue(tbl6.getColumn("k1").isKey());
         Assert.assertTrue(tbl6.getColumn("k2").isKey());
         Assert.assertTrue(tbl6.getColumn("k3").isKey());
 
-        OlapTable tbl7 = (OlapTable) db.getTable("tbl7");
+        OlapTable tbl7 = (OlapTable) db.getTableOrDdlException("tbl7");
         Assert.assertTrue(tbl7.getColumn("k1").isKey());
         Assert.assertFalse(tbl7.getColumn("k2").isKey());
         Assert.assertTrue(tbl7.getColumn("k2").getAggregationType() == AggregateType.NONE);
 
-        OlapTable tbl8 = (OlapTable) db.getTable("tbl8");
+        OlapTable tbl8 = (OlapTable) db.getTableOrDdlException("tbl8");
         Assert.assertTrue(tbl8.getColumn("k1").isKey());
         Assert.assertTrue(tbl8.getColumn("k2").isKey());
         Assert.assertFalse(tbl8.getColumn("v1").isKey());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
index 04c24bf..141e8e1 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
@@ -86,31 +86,31 @@ public class CreateViewTest {
                 () -> createView("create view test.view5 as select * from test.tbl1 where hour(now()) > 3" +
                         " and curdate() > '2021-06-26';"));
 
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database db = Catalog.getCurrentCatalog().getDbOrDdlException("default_cluster:test");
 
-        View view1 = (View) db.getTable("view1");
+        View view1 = (View) db.getTableOrDdlException("view1");
         Assert.assertEquals(4, view1.getFullSchema().size());
         Assert.assertNotNull(view1.getColumn("t1"));
         Assert.assertNotNull(view1.getColumn("t2"));
         Assert.assertNotNull(view1.getColumn("t3"));
         Assert.assertNotNull(view1.getColumn("t4"));
 
-        View view2 = (View) db.getTable("view2");
+        View view2 = (View) db.getTableOrDdlException("view2");
         Assert.assertEquals(4, view1.getFullSchema().size());
         Assert.assertNotNull(view2.getColumn("k1"));
         Assert.assertNotNull(view2.getColumn("k2"));
         Assert.assertNotNull(view2.getColumn("v1"));
         Assert.assertNotNull(view2.getColumn("v2"));
 
-        View view3 = (View) db.getTable("view3");
+        View view3 = (View) db.getTableOrDdlException("view3");
         Assert.assertEquals(1, view3.getFullSchema().size());
         Assert.assertNotNull(view3.getColumn("a1"));
 
-        View view4 = (View) db.getTable("view4");
+        View view4 = (View) db.getTableOrDdlException("view4");
         Assert.assertEquals(1, view4.getFullSchema().size());
         Assert.assertNotNull(view4.getColumn("s1"));
 
-        View view5 = (View) db.getTable("view5");
+        View view5 = (View) db.getTableOrDdlException("view5");
         System.out.println(view5.getDdlSql());
         Assert.assertTrue(view5.getDdlSql().contains("hour") && view5.getDdlSql().contains("now")
                 && view5.getDdlSql().contains("curdate"));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java
index ac49170..161d202 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DatabaseTest.java
@@ -27,7 +27,6 @@ import org.apache.doris.persist.EditLog;
 import org.apache.doris.thrift.TStorageType;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 
 import org.junit.Assert;
 import org.junit.Before;
@@ -41,7 +40,6 @@ import java.io.FileOutputStream;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import mockit.Expectations;
@@ -126,20 +124,20 @@ public class DatabaseTest {
         OlapTable table = new OlapTable(2000L, "baseTable", baseSchema, KeysType.AGG_KEYS,
                 new SinglePartitionInfo(), new RandomDistributionInfo(10));
         db.createTable(table);
-        Table resultTable1 = db.getTableOrThrowException(2000L, Table.TableType.OLAP);
-        Table resultTable2 = db.getTableOrThrowException("baseTable", Table.TableType.OLAP);
+        Table resultTable1 = db.getTableOrMetaException(2000L, Table.TableType.OLAP);
+        Table resultTable2 = db.getTableOrMetaException("baseTable", Table.TableType.OLAP);
         Assert.assertEquals(table, resultTable1);
         Assert.assertEquals(table, resultTable2);
         ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, tableId=3000",
-                () -> db.getTableOrThrowException(3000L, Table.TableType.OLAP));
-        ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, table=baseTable1",
-                () -> db.getTableOrThrowException("baseTable1", Table.TableType.OLAP));
+                () -> db.getTableOrMetaException(3000L, Table.TableType.OLAP));
+        ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class, "unknown table, tableName=baseTable1",
+                () -> db.getTableOrMetaException("baseTable1", Table.TableType.OLAP));
         ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class,
-                "table type is not BROKER, tableId=2000, type=class org.apache.doris.catalog.OlapTable",
-                () -> db.getTableOrThrowException(2000L, Table.TableType.BROKER));
+                "table type is not BROKER, tableId=2000, type=OLAP",
+                () -> db.getTableOrMetaException(2000L, Table.TableType.BROKER));
         ExceptionChecker.expectThrowsWithMsg(MetaNotFoundException.class,
-                "table type is not BROKER, table=baseTable, type=class org.apache.doris.catalog.OlapTable",
-                () -> db.getTableOrThrowException("baseTable", Table.TableType.BROKER));
+                "table type is not BROKER, tableName=baseTable, type=OLAP",
+                () -> db.getTableOrMetaException("baseTable", Table.TableType.BROKER));
     }
 
     @Test
@@ -159,8 +157,8 @@ public class DatabaseTest {
         // duplicate
         Assert.assertFalse(db.createTable(table));
 
-        Assert.assertEquals(table, db.getTable(table.getId()));
-        Assert.assertEquals(table, db.getTable(table.getName()));
+        Assert.assertEquals(table, db.getTableNullable(table.getId()));
+        Assert.assertEquals(table, db.getTableNullable(table.getName()));
 
         Assert.assertEquals(1, db.getTables().size());
         Assert.assertEquals(table, db.getTables().get(0));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
index 8a75300..d695fa5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
@@ -82,23 +82,23 @@ public class DropDbTest {
 
     @Test
     public void testNormalDropDb() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test1");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test1");
+        OlapTable table = (OlapTable) db.getTableOrMetaException("tbl1");
         Partition partition = table.getAllPartitions().iterator().next();
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         String dropDbSql = "drop database test1";
         dropDb(dropDbSql);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test1");
+        db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test1");
         List<Replica> replicaList = Catalog.getCurrentCatalog().getTabletInvertedIndex().getReplicasByTabletId(tabletId);
         Assert.assertNull(db);
         Assert.assertEquals(1, replicaList.size());
         String recoverDbSql = "recover database test1";
         RecoverDbStmt recoverDbStmt = (RecoverDbStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverDbSql, connectContext);
         Catalog.getCurrentCatalog().recoverDatabase(recoverDbStmt);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test1");
+        db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test1");
         Assert.assertNotNull(db);
         Assert.assertEquals("default_cluster:test1", db.getFullName());
-        table = (OlapTable) db.getTable("tbl1");
+        table = (OlapTable) db.getTableOrMetaException("tbl1");
         Assert.assertNotNull(table);
         Assert.assertEquals("tbl1", table.getName());
     }
@@ -106,12 +106,12 @@ public class DropDbTest {
     @Test
     public void testForceDropDb() throws Exception {
         String dropDbSql = "drop database test2 force";
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test2");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test2");
+        OlapTable table = (OlapTable) db.getTableOrMetaException("tbl1");
         Partition partition = table.getAllPartitions().iterator().next();
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         dropDb(dropDbSql);
-        db = Catalog.getCurrentCatalog().getDb("default_cluster:test2");
+        db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test2");
         List<Replica> replicaList = Catalog.getCurrentCatalog().getTabletInvertedIndex().getReplicasByTabletId(tabletId);
         Assert.assertNull(db);
         Assert.assertTrue(replicaList.isEmpty());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
index ba2ef86..43f18ce 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
@@ -80,8 +80,8 @@ public class DropPartitionTest {
 
     @Test
     public void testNormalDropPartition() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         Partition partition = table.getPartition("p20210201");
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         String dropPartitionSql = " alter table test.tbl1 drop partition p20210201;";
@@ -100,8 +100,8 @@ public class DropPartitionTest {
 
     @Test
     public void testForceDropPartition() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         Partition partition = table.getPartition("p20210202");
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         String dropPartitionSql = " alter table test.tbl1 drop partition p20210202 force;";
@@ -119,8 +119,8 @@ public class DropPartitionTest {
 
     @Test
     public void testDropPartitionAndReserveTablets() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         Partition partition = table.getPartition("p20210203");
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         table.dropPartitionAndReserveTablet("p20210203");
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
index 4c12e53..fc543fe 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
@@ -79,8 +79,8 @@ public class DropTableTest {
 
     @Test
     public void testNormalDropTable() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("tbl1");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTableOrMetaException("tbl1");
         Partition partition = table.getAllPartitions().iterator().next();
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         String dropTableSql = "drop table test.tbl1";
@@ -90,15 +90,15 @@ public class DropTableTest {
         String recoverDbSql = "recover table test.tbl1";
         RecoverTableStmt recoverTableStmt = (RecoverTableStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverDbSql, connectContext);
         Catalog.getCurrentCatalog().recoverTable(recoverTableStmt);
-        table = (OlapTable) db.getTable("tbl1");
+        table = (OlapTable) db.getTableOrMetaException("tbl1");
         Assert.assertNotNull(table);
         Assert.assertEquals("tbl1", table.getName());
     }
 
     @Test
     public void testForceDropTable() throws Exception {
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("tbl2");
+        Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTableOrMetaException("tbl2");
         Partition partition = table.getAllPartitions().iterator().next();
         long tabletId = partition.getBaseIndex().getTablets().get(0).getId();
         String dropTableSql = "drop table test.tbl2 force";
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
index 754a44e..95852ae 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
@@ -116,8 +116,8 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable("dynamic_partition_normal");
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTableOrAnalysisException("dynamic_partition_normal");
         Assert.assertEquals(table.getTableProperty().getDynamicPartitionProperty().getReplicationNum(), DynamicPartitionProperty.NOT_SET_REPLICATION_NUM);
     }
 
@@ -456,8 +456,8 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.replication_num\" = \"2\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable table = (OlapTable) db.getTable(tableName);
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
+        OlapTable table = (OlapTable) db.getTableOrAnalysisException(tableName);
         Assert.assertEquals(table.getTableProperty().getDynamicPartitionProperty().getReplicationNum(), 2);
     }
 
@@ -485,7 +485,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("empty_dynamic_partition");
+        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("empty_dynamic_partition");
         Assert.assertTrue(emptyDynamicTable.getAllPartitions().size() == 4);
 
         Iterator<Partition> partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -539,7 +539,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("histo_dynamic_partition");
+        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("histo_dynamic_partition");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         Iterator<Partition> partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -676,8 +676,8 @@ public class DynamicPartitionTableTest {
                 ");";
         // start and history_partition_num are set, create ok
         ExceptionChecker.expectThrowsNoException(() -> createTable(createOlapTblStmt4));
-        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
-        OlapTable tbl = (OlapTable) db.getTable("dynamic_partition3");
+        Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrAnalysisException("dynamic_partition3");
         Assert.assertEquals(9, tbl.getPartitionNames().size());
 
         // alter dynamic partition property of table dynamic_partition3
@@ -720,7 +720,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("history_dynamic_partition_day");
+        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("history_dynamic_partition_day");
         Map<String, String> tableProperties = emptyDynamicTable.getTableProperty().getProperties();
         Assert.assertEquals(14, emptyDynamicTable.getAllPartitions().size());
         // never delete the old partitions
@@ -747,7 +747,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("hour_dynamic_partition");
+        OlapTable emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("hour_dynamic_partition");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         Iterator<Partition> partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -774,7 +774,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("week_dynamic_partition");
+        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("week_dynamic_partition");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -801,7 +801,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("month_dynamic_partition");
+        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("month_dynamic_partition");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -828,7 +828,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("int_dynamic_partition_day");
+        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("int_dynamic_partition_day");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -855,7 +855,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("int_dynamic_partition_week");
+        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("int_dynamic_partition_week");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -882,7 +882,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.buckets\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDb("default_cluster:test").getTable("int_dynamic_partition_month");
+        emptyDynamicTable = (OlapTable) Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test").getTableOrAnalysisException("int_dynamic_partition_month");
         Assert.assertEquals(7, emptyDynamicTable.getAllPartitions().size());
 
         partitionIterator = emptyDynamicTable.getAllPartitions().iterator();
@@ -916,7 +916,7 @@ public class DynamicPartitionTableTest {
 
     @Test
     public void testHotPartitionNum() throws Exception {
-        Database testDb = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Database testDb = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
         // 1. hour
         String createOlapTblStmt = "CREATE TABLE test.`hot_partition_hour_tbl1` (\n" +
                 "  `k1` datetime NULL COMMENT \"\",\n" +
@@ -937,7 +937,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        OlapTable tbl = (OlapTable)testDb.getTable("hot_partition_hour_tbl1");
+        OlapTable tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_hour_tbl1");
         RangePartitionInfo partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         Map<Long, DataProperty> idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(7, idToDataProperty.size());
@@ -970,7 +970,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"0\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_hour_tbl2");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_hour_tbl2");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(7, idToDataProperty.size());
@@ -997,7 +997,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"3\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_hour_tbl3");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_hour_tbl3");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(7, idToDataProperty.size());
@@ -1030,7 +1030,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"2\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_day_tbl1");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_day_tbl1");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(4, idToDataProperty.size());
@@ -1057,7 +1057,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"2\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_day_tbl2");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_day_tbl2");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(8, idToDataProperty.size());
@@ -1090,7 +1090,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"1\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_week_tbl1");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_week_tbl1");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(8, idToDataProperty.size());
@@ -1123,7 +1123,7 @@ public class DynamicPartitionTableTest {
                 "\"dynamic_partition.hot_partition_num\" = \"4\"\n" +
                 ");";
         createTable(createOlapTblStmt);
-        tbl = (OlapTable)testDb.getTable("hot_partition_month_tbl1");
+        tbl = (OlapTable)testDb.getTableOrAnalysisException("hot_partition_month_tbl1");
         partitionInfo = (RangePartitionInfo) tbl.getPartitionInfo();
         idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty);
         Assert.assertEquals(8, idToDataProperty.size());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java
index 1cf7fdd..0a808c9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/InfoSchemaDbTest.java
@@ -31,6 +31,6 @@ public class InfoSchemaDbTest {
         Assert.assertFalse(db.createTableWithLock(null, false, false).first);
         db.dropTable("authors");
         db.write(null);
-        Assert.assertNull(db.getTable("authors"));
+        Assert.assertNull(db.getTableNullable("authors"));
     }
 }
\ No newline at end of file
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/MetadataViewerTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/MetadataViewerTest.java
index bb348f1..d8694b5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/MetadataViewerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/MetadataViewerTest.java
@@ -21,7 +21,6 @@ import org.apache.doris.analysis.BinaryPredicate.Operator;
 import org.apache.doris.analysis.PartitionNames;
 import org.apache.doris.backup.CatalogMocker;
 import org.apache.doris.catalog.Replica.ReplicaStatus;
-import org.apache.doris.common.AnalysisException;
 import org.apache.doris.system.SystemInfoService;
 
 import com.google.common.collect.Lists;
@@ -52,8 +51,7 @@ public class MetadataViewerTest {
     private static Database db;
 
     @BeforeClass
-    public static void setUp() throws NoSuchMethodException, SecurityException, InstantiationException,
-            IllegalAccessException, IllegalArgumentException, InvocationTargetException, AnalysisException {
+    public static void setUp() throws Exception {
         Class[] argTypes = new Class[] { String.class, String.class, List.class, ReplicaStatus.class, Operator.class };
         getTabletStatusMethod = MetadataViewer.class.getDeclaredMethod("getTabletStatus", argTypes);
         getTabletStatusMethod.setAccessible(true);
@@ -66,7 +64,7 @@ public class MetadataViewerTest {
     }
 
     @Before
-    public void before() {
+    public void before() throws Exception {
 
         new Expectations() {
             {
@@ -74,7 +72,7 @@ public class MetadataViewerTest {
                 minTimes = 0;
                 result = catalog;
 
-                catalog.getDb(anyString);
+                catalog.getDbOrDdlException(anyString);
                 minTimes = 0;
                 result = db;
             }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
index e2ca0e8..a69762d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
@@ -103,37 +103,21 @@ public class RecoverTest {
     }
 
     private static boolean checkDbExist(String dbName) {
-        Database db = Catalog.getCurrentCatalog().getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName));
-        return db != null;
+        return Catalog.getCurrentCatalog().getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName)).isPresent();
     }
 
     private static boolean checkTableExist(String dbName, String tblName) {
-        Database db = Catalog.getCurrentCatalog().getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName));
-        if (db == null) {
-            return false;
-        }
-
-        Table tbl = db.getTable(tblName);
-        return tbl != null;
+        return Catalog.getCurrentCatalog()
+                .getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName))
+                .flatMap(db -> db.getTable(tblName)).isPresent();
     }
 
     private static boolean checkPartitionExist(String dbName, String tblName, String partName) {
-        Database db = Catalog.getCurrentCatalog().getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName));
-        if (db == null) {
-            return false;
-        }
-
-        Table tbl = db.getTable(tblName);
-        if (tbl == null) {
-            return false;
-        }
-
-        Partition partition = tbl.getPartition(partName);
-        return partition != null;
+        return Catalog.getCurrentCatalog()
+                .getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName))
+                .flatMap(db -> db.getTable(tblName)).map(table -> table.getPartition(partName)).isPresent();
     }
 
-
-
     @Test
     public void testRecover() throws Exception {
         createDb("test");
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
index 268ef36..56a0bde 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
@@ -221,8 +221,8 @@ public class TempPartitionTest {
         CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
         Catalog.getCurrentCatalog().createTable(createTableStmt);
 
-        Database db2 = Catalog.getCurrentCatalog().getDb("default_cluster:db2");
-        OlapTable tbl2 = (OlapTable) db2.getTable("tbl2");
+        Database db2 = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:db2");
+        OlapTable tbl2 = (OlapTable) db2.getTableOrAnalysisException("tbl2");
 
         testSerializeOlapTable(tbl2);
 
@@ -508,8 +508,8 @@ public class TempPartitionTest {
         CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
         Catalog.getCurrentCatalog().createTable(createTableStmt);
 
-        Database db3 = Catalog.getCurrentCatalog().getDb("default_cluster:db3");
-        OlapTable tbl3 = (OlapTable) db3.getTable("tbl3");
+        Database db3 = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:db3");
+        OlapTable tbl3 = (OlapTable) db3.getTableOrAnalysisException("tbl3");
 
         // base range is [min, 10), [10, 20), [20, 30)
 
@@ -584,8 +584,8 @@ public class TempPartitionTest {
         CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
         Catalog.getCurrentCatalog().createTable(createTableStmt);
 
-        Database db4 = Catalog.getCurrentCatalog().getDb("default_cluster:db4");
-        OlapTable tbl4 = (OlapTable) db4.getTable("tbl4");
+        Database db4 = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:db4");
+        OlapTable tbl4 = (OlapTable) db4.getTableOrAnalysisException("tbl4");
 
         testSerializeOlapTable(tbl4);
 
@@ -922,8 +922,8 @@ public class TempPartitionTest {
         CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
         Catalog.getCurrentCatalog().createTable(createTableStmt);
 
-        Database db5 = Catalog.getCurrentCatalog().getDb("default_cluster:db5");
-        OlapTable tbl5 = (OlapTable) db5.getTable("tbl5");
+        Database db5 = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:db5");
+        OlapTable tbl5 = (OlapTable) db5.getTableOrAnalysisException("tbl5");
 
         testSerializeOlapTable(tbl5);
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
index a6ea1e4..537fefc 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
@@ -62,6 +62,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
 
@@ -83,7 +84,7 @@ public class RebalanceTest {
     private Map<String, ClusterLoadStatistic> statisticMap;
 
     @Before
-    public void setUp() throws AnalysisException {
+    public void setUp() throws Exception {
         db = new Database(1, "test db");
         db.setClusterName(SystemInfoService.DEFAULT_CLUSTER);
         new Expectations() {
@@ -92,7 +93,11 @@ public class RebalanceTest {
                 minTimes = 0;
                 result = db.getId();
 
-                catalog.getDb(anyLong);
+                catalog.getDbNullable(anyLong);
+                minTimes = 0;
+                result = db;
+
+                catalog.getDbOrException(anyLong, (Function<Long, SchedException>) any);
                 minTimes = 0;
                 result = db;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
index aa4a970..6a24b2a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
@@ -93,11 +93,11 @@ public class SystemInfoServiceTest {
                 minTimes = 0;
                 result = editLog;
 
-                catalog.getDb(anyLong);
+                catalog.getDbNullable(anyLong);
                 minTimes = 0;
                 result = db;
 
-                db.getTable(anyLong);
+                db.getTableNullable(anyLong);
                 minTimes = 0;
                 result = table;
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/proc/DbsProcDirTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/proc/DbsProcDirTest.java
index aa9930a..fcc5fa6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/common/proc/DbsProcDirTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/common/proc/DbsProcDirTest.java
@@ -68,27 +68,27 @@ public class DbsProcDirTest {
     public void testLookupNormal() throws AnalysisException {
         new Expectations(catalog) {
             {
-                catalog.getDb("db1");
+                catalog.getDbNullable("db1");
                 minTimes = 0;
                 result = db1;
... 1302 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org