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 2022/06/13 02:55:48 UTC

[incubator-doris] branch master updated: [feature](multi-catalog) Change DatabaseIf APIs' return type to TableIf. (#10044)

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 f26b81e4dd [feature](multi-catalog) Change DatabaseIf APIs' return type to TableIf. (#10044)
f26b81e4dd is described below

commit f26b81e4dd6482c304d221fc85e8410a695c1309
Author: Jibing-Li <64...@users.noreply.github.com>
AuthorDate: Mon Jun 13 10:55:44 2022 +0800

    [feature](multi-catalog) Change DatabaseIf APIs' return type to TableIf. (#10044)
    
    Change the DatabaseIf APIs' return type to TableIf.
    Use generics in DatabaseIf, to avoid changing the return type in Database.
    Currently Database class use type Table, I'm try to avoid changing it to TableIf.
    Because in this case, we need to change a lot of code.
---
 .../main/java/org/apache/doris/alter/Alter.java    | 14 ++--
 .../java/org/apache/doris/alter/AlterHandler.java  |  2 +-
 .../java/org/apache/doris/alter/AlterJobV2.java    |  2 +-
 .../doris/alter/MaterializedViewHandler.java       |  6 +-
 .../java/org/apache/doris/alter/RollupJobV2.java   | 10 +--
 .../apache/doris/alter/SchemaChangeHandler.java    |  6 +-
 .../org/apache/doris/alter/SchemaChangeJobV2.java  | 10 +--
 .../org/apache/doris/analysis/ShowDataStmt.java    |  2 +-
 .../java/org/apache/doris/catalog/Catalog.java     | 14 ++--
 .../apache/doris/catalog/ColocateTableIndex.java   |  3 +-
 .../java/org/apache/doris/catalog/Database.java    | 10 +--
 .../java/org/apache/doris/catalog/DatabaseIf.java  | 41 +++++-----
 .../org/apache/doris/catalog/InternalDatabase.java | 91 ++++++++++++++++++++++
 .../main/java/org/apache/doris/catalog/Table.java  |  2 +-
 .../doris/catalog/external/ExternalDatabase.java   | 46 ++++++-----
 .../doris/datasource/InternalDataSource.java       | 14 ++--
 .../apache/doris/httpv2/rest/RowCountAction.java   |  2 +-
 .../doris/httpv2/rest/TableRowCountAction.java     |  2 +-
 .../doris/httpv2/rest/TableSchemaAction.java       |  2 +-
 .../src/main/java/org/apache/doris/load/Load.java  |  2 +-
 .../doris/load/routineload/RoutineLoadJob.java     |  3 +-
 .../doris/load/routineload/RoutineLoadManager.java |  2 +-
 .../org/apache/doris/master/ReportHandler.java     |  2 +-
 .../org/apache/doris/alter/AlterJobV2Test.java     |  2 +-
 .../java/org/apache/doris/alter/AlterTest.java     |  2 +-
 .../apache/doris/catalog/DropPartitionTest.java    |  6 +-
 .../org/apache/doris/utframe/AnotherDemoTest.java  |  2 +-
 .../java/org/apache/doris/utframe/DemoTest.java    |  4 +-
 28 files changed, 200 insertions(+), 104 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 4857ef7d90..a22e7e896f 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
@@ -115,7 +115,7 @@ public class Alter {
         // check db quota
         db.checkQuota();
 
-        OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP);
         ((MaterializedViewHandler) materializedViewHandler).processCreateMaterializedView(stmt, db, olapTable);
     }
 
@@ -125,7 +125,7 @@ public class Alter {
         Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         String tableName = stmt.getTableName().getTbl();
-        OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP);
         // drop materialized view
         ((MaterializedViewHandler) materializedViewHandler).processDropMaterializedView(stmt, db, olapTable);
     }
@@ -469,8 +469,8 @@ public class Alter {
         long newTblId = log.getNewTblId();
 
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable origTable = db.getTableOrMetaException(origTblId, TableType.OLAP);
-        OlapTable newTbl = db.getTableOrMetaException(newTblId, TableType.OLAP);
+        OlapTable origTable = (OlapTable) db.getTableOrMetaException(origTblId, TableType.OLAP);
+        OlapTable newTbl = (OlapTable) db.getTableOrMetaException(newTblId, TableType.OLAP);
         List<Table> tableList = Lists.newArrayList(origTable, newTbl);
         tableList.sort((Comparator.comparing(Table::getId)));
         MetaLockUtils.writeLockTablesOrMetaException(tableList);
@@ -529,7 +529,7 @@ public class Alter {
         Database db = Catalog.getCurrentCatalog().getDbOrDdlException(dbName);
 
         String tableName = dbTableName.getTbl();
-        View view = db.getTableOrMetaException(tableName, TableType.VIEW);
+        View view = (View) db.getTableOrMetaException(tableName, TableType.VIEW);
         modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns());
     }
 
@@ -567,7 +567,7 @@ public class Alter {
         List<Column> newFullSchema = alterViewInfo.getNewFullSchema();
 
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        View view = db.getTableOrMetaException(tableId, TableType.VIEW);
+        View view = (View) db.getTableOrMetaException(tableId, TableType.VIEW);
 
         db.writeLock();
         view.writeLock();
@@ -717,7 +717,7 @@ public class Alter {
 
     public void replayModifyPartition(ModifyPartitionInfo info) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable 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 23a66a3678..5205d418b3 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
@@ -197,7 +197,7 @@ public abstract class AlterHandler extends MasterDaemon {
     public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(task.getDbId());
 
-        OlapTable tbl = db.getTableOrMetaException(task.getTableId(), Table.TableType.OLAP);
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException(task.getTableId(), Table.TableType.OLAP);
         tbl.writeLockOrMetaException();
         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 cb58707c1a..a69ad6266a 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
@@ -186,7 +186,7 @@ public abstract class AlterJobV2 implements Writable {
     protected boolean checkTableStable(Database db) throws AlterCancelException {
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
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 2c8e3dc550..66465e0aac 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
@@ -831,7 +831,7 @@ public class MaterializedViewHandler extends AlterHandler {
 
         TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
         Database db = catalog.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         olapTable.writeLock();
         try {
             for (Partition partition : olapTable.getPartitions()) {
@@ -878,7 +878,7 @@ public class MaterializedViewHandler extends AlterHandler {
     private void changeTableStatus(long dbId, long tableId, OlapTableState olapTableState) {
         try {
             Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-            OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
             olapTable.writeLockOrMetaException();
             try {
                 if (olapTable.getState() == olapTableState) {
@@ -1035,7 +1035,7 @@ public class MaterializedViewHandler extends AlterHandler {
         List<AlterJobV2> rollupJobV2List = new ArrayList<>();
         OlapTable olapTable;
         try {
-            olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+            olapTable = (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/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
index 7f3a115ed4..45acf8345c 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
@@ -195,7 +195,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
         MarkedCountDownLatch<Long, Long> countDownLatch = new MarkedCountDownLatch<Long, Long>(totalReplicaNum);
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -333,7 +333,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
 
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -404,7 +404,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
 
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -542,7 +542,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
      */
     private void replayCreateJob(RollupJobV2 replayedJob) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
 
         olapTable.writeLock();
         try {
@@ -582,7 +582,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
      */
     private void replayPendingJob(RollupJobV2 replayedJob) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         olapTable.writeLock();
         try {
             addRollupIndexToCatalog(olapTable);
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 46c65acf21..1ffa5b8607 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
@@ -1617,7 +1617,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 = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         olapTable.readLock();
         try {
             partitions.addAll(olapTable.getPartitions());
@@ -1649,7 +1649,7 @@ public class SchemaChangeHandler extends AlterHandler {
                                              String tableName,
                                              List<String> partitionNames,
                                              Map<String, String> properties) throws DdlException, MetaNotFoundException {
-        OlapTable olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY));
         if (isInMemory == olapTable.isInMemory()) {
             return;
@@ -1676,7 +1676,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 = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         olapTable.readLock();
         try {
             Partition partition = olapTable.getPartition(partitionName);
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 a7faa58743..d73f22617d 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
@@ -212,7 +212,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
 
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -367,7 +367,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
 
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -439,7 +439,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
 
         OlapTable tbl;
         try {
-            tbl = db.getTableOrMetaException(tableId, TableType.OLAP);
+            tbl = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         } catch (MetaNotFoundException e) {
             throw new AlterCancelException(e.getMessage());
         }
@@ -652,7 +652,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
      */
     private void replayCreateJob(SchemaChangeJobV2 replayedJob) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
@@ -690,7 +690,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
      */
     private void replayPendingJob(SchemaChangeJobV2 replayedJob) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             addShadowIndexToCatalog(olapTable);
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 3f8637b762..22f42448cc 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
@@ -227,7 +227,7 @@ public class ShowDataStmt extends ShowStmt {
                         dbName + ": " + tableName);
             }
 
-            OlapTable olapTable = db.getTableOrMetaException(tableName, TableType.OLAP);
+            OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP);
             long totalSize = 0;
             long totalReplicaCount = 0;
 
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 d051e8afbf..7050344beb 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
@@ -3800,7 +3800,7 @@ public class Catalog {
         Map<String, String> properties = info.getPropertyMap();
 
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             modifyTableColocate(db, olapTable, properties.get(PropertyAnalyzer.PROPERTIES_COLOCATE_WITH), true,
@@ -3860,7 +3860,7 @@ public class Catalog {
         String newRollupName = tableInfo.getNewRollupName();
 
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             String rollupName = olapTable.getIndexNameById(indexId);
@@ -3921,7 +3921,7 @@ public class Catalog {
         String newPartitionName = tableInfo.getNewPartitionName();
 
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = olapTable.getPartition(partitionId);
@@ -4063,7 +4063,7 @@ public class Catalog {
         Map<String, String> properties = info.getProperties();
 
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             TableProperty tableProperty = olapTable.getTableProperty();
@@ -4138,7 +4138,7 @@ public class Catalog {
         int bucketNum = info.getBucketNum();
 
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             DistributionInfo defaultDistributionInfo = olapTable.getDefaultDistributionInfo();
@@ -4613,7 +4613,7 @@ public class Catalog {
 
     public void replayConvertDistributionType(TableInfo info) throws MetaNotFoundException {
         Database db = this.getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             olapTable.convertHashDistributionToRandomDistribution();
@@ -4657,7 +4657,7 @@ public class Catalog {
         long dbId = replaceTempPartitionLog.getDbId();
         long tableId = replaceTempPartitionLog.getTblId();
         Database db = this.getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
         olapTable.writeLock();
         try {
             olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(),
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 8f508466fb..b77890b8e7 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
@@ -522,7 +522,8 @@ public class ColocateTableIndex implements Writable {
 
     public void replayAddTableToGroup(ColocatePersistInfo info) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getGroupId().dbId);
-        OlapTable tbl = db.getTableOrMetaException(info.getTableId(), org.apache.doris.catalog.Table.TableType.OLAP);
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException(info.getTableId(),
+                org.apache.doris.catalog.Table.TableType.OLAP);
         writeLock();
         try {
             Map<Tag, List<List<Long>>> map = info.getBackendsPerBucketSeq();
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 bffcb41263..941b005d4f 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
@@ -71,7 +71,7 @@ import javax.annotation.Nullable;
  * if the table has never been loaded * if the table loading failed on the
  * previous attempt
  */
-public class Database extends MetaObject implements Writable, DatabaseIf {
+public class Database extends MetaObject implements Writable, InternalDatabase<Table> {
     private static final Logger LOG = LogManager.getLogger(Database.class);
 
     private long id;
@@ -509,21 +509,21 @@ public class Database extends MetaObject implements Writable, DatabaseIf {
     }
 
     @SuppressWarnings("unchecked")
-    public <T extends Table> T getTableOrMetaException(String tableName, TableType tableType) throws MetaNotFoundException {
+    public Table 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;
+        return table;
     }
 
     @SuppressWarnings("unchecked")
-    public <T extends Table> T getTableOrMetaException(long tableId, TableType tableType) throws MetaNotFoundException {
+    public Table 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.getType());
         }
-        return (T) table;
+        return table;
     }
 
     public Table getTableOrDdlException(String tableName) throws DdlException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java
index a34912150c..840e091c82 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java
@@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit;
  * I just copied some common interface from the origin Database class.
  * Maybe changed later.
  */
-public interface DatabaseIf {
+public interface DatabaseIf<T extends TableIf> {
 
     void readLock();
 
@@ -60,47 +60,46 @@ public interface DatabaseIf {
 
     boolean isTableExist(String tableName);
 
-    List<Table> getTables();
+    List<T> getTables();
 
-    List<Table> getTablesOnIdOrder();
+    List<T> getTablesOnIdOrder();
 
-    List<Table> getViews();
+    List<T> getViews();
 
-    List<Table> getTablesOnIdOrderIfExist(List<Long> tableIdList);
+    List<T> getTablesOnIdOrderIfExist(List<Long> tableIdList);
 
-    List<Table> getTablesOnIdOrderOrThrowException(List<Long> tableIdList) throws MetaNotFoundException;
+    List<T> getTablesOnIdOrderOrThrowException(List<Long> tableIdList) throws MetaNotFoundException;
 
     Set<String> getTableNamesWithLock();
 
-    Table getTableNullable(String tableName);
+    T getTableNullable(String tableName);
 
-    Optional<Table> getTable(String tableName);
+    Optional<T> getTable(String tableName);
 
-    Optional<Table> getTable(long tableId);
+    Optional<T> getTable(long tableId);
 
-    <E extends Exception> Table getTableOrException(String tableName, java.util.function.Function<String, E> e)
-            throws E;
+    <E extends Exception> T getTableOrException(String tableName, java.util.function.Function<String, E> e) throws E;
 
-    <E extends Exception> Table getTableOrException(long tableId, java.util.function.Function<Long, E> e) throws E;
+    <E extends Exception> T getTableOrException(long tableId, java.util.function.Function<Long, E> e) throws E;
 
-    Table getTableOrMetaException(String tableName) throws MetaNotFoundException;
+    T getTableOrMetaException(String tableName) throws MetaNotFoundException;
 
-    Table getTableOrMetaException(long tableId) throws MetaNotFoundException;
+    T getTableOrMetaException(long tableId) throws MetaNotFoundException;
 
     @SuppressWarnings("unchecked")
-    <T extends Table> T getTableOrMetaException(String tableName, Table.TableType tableType)
+    <V extends Table> V getTableOrMetaException(String tableName, Table.TableType tableType)
             throws MetaNotFoundException;
 
     @SuppressWarnings("unchecked")
-    <T extends Table> T getTableOrMetaException(long tableId, Table.TableType tableType) throws MetaNotFoundException;
+    <V extends Table> V getTableOrMetaException(long tableId, Table.TableType tableType) throws MetaNotFoundException;
 
-    Table getTableOrDdlException(String tableName) throws DdlException;
+    T getTableOrDdlException(String tableName) throws DdlException;
 
-    Table getTableOrDdlException(long tableId) throws DdlException;
+    T getTableOrDdlException(long tableId) throws DdlException;
 
-    Table getTableOrAnalysisException(String tableName) throws AnalysisException;
+    T getTableOrAnalysisException(String tableName) throws AnalysisException;
 
-    OlapTable getOlapTableOrAnalysisException(String tableName) throws AnalysisException;
+    T getTableOrAnalysisException(long tableId) throws AnalysisException;
 
-    Table getTableOrAnalysisException(long tableId) throws AnalysisException;
+    OlapTable getOlapTableOrAnalysisException(String tableName) throws AnalysisException;
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalDatabase.java
new file mode 100644
index 0000000000..5dab055268
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/InternalDatabase.java
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.catalog;
+
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.MetaNotFoundException;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.function.Function;
+
+/**
+ * InternalDatabase interface for the current Database class.
+ *
+ * @param <T> current Database table type, subclass of Table.
+ */
+public interface InternalDatabase<T extends Table> extends DatabaseIf<T> {
+
+    @Override
+    public List<T> getTables();
+
+    @Override
+    public List<T> getTablesOnIdOrder();
+
+    @Override
+    public List<T> getViews();
+
+    @Override
+    public List<T> getTablesOnIdOrderIfExist(List<Long> tableIdList);
+
+    @Override
+    public List<T> getTablesOnIdOrderOrThrowException(List<Long> tableIdList) throws MetaNotFoundException;
+
+    @Override
+    public T getTableNullable(String tableName);
+
+    @Override
+    public Optional<T> getTable(String tableName);
+
+    @Override
+    public Optional<T> getTable(long tableId);
+
+    @Override
+    public <E extends Exception> T getTableOrException(String tableName, Function<String, E> e) throws E;
+
+    @Override
+    public <E extends Exception> T getTableOrException(long tableId, Function<Long, E> e) throws E;
+
+    @Override
+    public T getTableOrMetaException(String tableName) throws MetaNotFoundException;
+
+    @Override
+    public T getTableOrMetaException(long tableId) throws MetaNotFoundException;
+
+    @Override
+    public T getTableOrMetaException(String tableName, Table.TableType tableType) throws MetaNotFoundException;
+
+    @Override
+    public T getTableOrMetaException(long tableId, Table.TableType tableType) throws MetaNotFoundException;
+
+    @Override
+    public T getTableOrDdlException(String tableName) throws DdlException;
+
+    @Override
+    public T getTableOrDdlException(long tableId) throws DdlException;
+
+    @Override
+    public T getTableOrAnalysisException(String tableName) throws AnalysisException;
+
+    @Override
+    public T getTableOrAnalysisException(long tableId) throws AnalysisException;
+
+    @Override
+    public OlapTable getOlapTableOrAnalysisException(String tableName) throws AnalysisException;
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
index df70948dcd..0ffbadcdc1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
@@ -49,7 +49,7 @@ import java.util.stream.Collectors;
 /**
  * Internal representation of table-related metadata. A table contains several partitions.
  */
-public class Table extends MetaObject implements Writable {
+public class Table extends MetaObject implements Writable, TableIf {
     private static final Logger LOG = LogManager.getLogger(Table.class);
 
     // empirical value.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
index 81f51de688..e193c33dcd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.catalog.external;
 
-import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.DatabaseProperty;
 import org.apache.doris.catalog.OlapTable;
@@ -38,9 +37,14 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 
-public class ExternalDatabase implements DatabaseIf {
+/**
+ * Base class of external database.
+ *
+ * @param <T> External table type is ExternalTable or its subclass.
+ */
+public class ExternalDatabase<T extends ExternalTable> implements DatabaseIf<T> {
 
-    private static final Logger LOG = LogManager.getLogger(Database.class);
+    private static final Logger LOG = LogManager.getLogger(ExternalDatabase.class);
 
     private long id;
     private String name;
@@ -118,27 +122,27 @@ public class ExternalDatabase implements DatabaseIf {
     }
 
     @Override
-    public List<Table> getTables() {
+    public List<T> getTables() {
         return null;
     }
 
     @Override
-    public List<Table> getTablesOnIdOrder() {
+    public List<T> getTablesOnIdOrder() {
         return null;
     }
 
     @Override
-    public List<Table> getViews() {
+    public List<T> getViews() {
         return null;
     }
 
     @Override
-    public List<Table> getTablesOnIdOrderIfExist(List<Long> tableIdList) {
+    public List<T> getTablesOnIdOrderIfExist(List<Long> tableIdList) {
         return null;
     }
 
     @Override
-    public List<Table> getTablesOnIdOrderOrThrowException(List<Long> tableIdList) throws MetaNotFoundException {
+    public List<T> getTablesOnIdOrderOrThrowException(List<Long> tableIdList) throws MetaNotFoundException {
         return null;
     }
 
@@ -148,64 +152,64 @@ public class ExternalDatabase implements DatabaseIf {
     }
 
     @Override
-    public Table getTableNullable(String tableName) {
+    public T getTableNullable(String tableName) {
         return null;
     }
 
     @Override
-    public Optional<Table> getTable(String tableName) {
+    public Optional<T> getTable(String tableName) {
         return Optional.empty();
     }
 
     @Override
-    public Optional<Table> getTable(long tableId) {
+    public Optional<T> getTable(long tableId) {
         return Optional.empty();
     }
 
     @Override
-    public <E extends Exception> Table getTableOrException(String tableName, Function<String, E> e) throws E {
+    public <E extends Exception> T getTableOrException(String tableName, Function<String, E> e) throws E {
         return null;
     }
 
     @Override
-    public <E extends Exception> Table getTableOrException(long tableId, Function<Long, E> e) throws E {
+    public <E extends Exception> T getTableOrException(long tableId, Function<Long, E> e) throws E {
         return null;
     }
 
     @Override
-    public Table getTableOrMetaException(String tableName) throws MetaNotFoundException {
+    public T getTableOrMetaException(String tableName) throws MetaNotFoundException {
         return null;
     }
 
     @Override
-    public Table getTableOrMetaException(long tableId) throws MetaNotFoundException {
+    public T getTableOrMetaException(long tableId) throws MetaNotFoundException {
         return null;
     }
 
     @Override
-    public <T extends Table> T getTableOrMetaException(String tableName, Table.TableType tableType)
+    public <V extends Table> V getTableOrMetaException(String tableName, Table.TableType tableType)
             throws MetaNotFoundException {
         return null;
     }
 
     @Override
-    public <T extends Table> T getTableOrMetaException(long tableId, Table.TableType tableType)
+    public <V extends Table> V getTableOrMetaException(long tableId, Table.TableType tableType)
             throws MetaNotFoundException {
         return null;
     }
 
     @Override
-    public Table getTableOrDdlException(String tableName) throws DdlException {
+    public T getTableOrDdlException(String tableName) throws DdlException {
         return null;
     }
 
     @Override
-    public Table getTableOrDdlException(long tableId) throws DdlException {
+    public T getTableOrDdlException(long tableId) throws DdlException {
         return null;
     }
 
     @Override
-    public Table getTableOrAnalysisException(String tableName) throws AnalysisException {
+    public T getTableOrAnalysisException(String tableName) throws AnalysisException {
         return null;
     }
 
@@ -215,7 +219,7 @@ public class ExternalDatabase implements DatabaseIf {
     }
 
     @Override
-    public Table getTableOrAnalysisException(long tableId) throws AnalysisException {
+    public T getTableOrAnalysisException(long tableId) throws AnalysisException {
         return null;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
index 8395ebbd63..dcacc806a1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
@@ -992,7 +992,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayAddReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             unprotectAddReplica(olapTable, info);
@@ -1003,7 +1003,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayUpdateReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             unprotectUpdateReplica(olapTable, info);
@@ -1021,7 +1021,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayDeleteReplica(ReplicaPersistInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             unprotectDeleteReplica(olapTable, info);
@@ -1451,7 +1451,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayAddPartition(PartitionPersistInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = info.getPartition();
@@ -1550,7 +1550,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayDropPartition(DropPartitionInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             if (info.isTempPartition()) {
@@ -1569,7 +1569,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayRecoverPartition(RecoverInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             Catalog.getCurrentRecycleBin().replayRecoverPartition(olapTable, info.getPartitionId());
@@ -2470,7 +2470,7 @@ public class InternalDataSource implements DataSourceIf {
 
     public void replayTruncateTable(TruncateTableInfo info) throws MetaNotFoundException {
         Database db = (Database) getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTblId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTblId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable());
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 b9dfa1d1ec..75510386fd 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
@@ -68,7 +68,7 @@ public class RowCountAction extends RestBaseController {
         OlapTable olapTable;
         try {
             Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
-            olapTable = db.getTableOrMetaException(tableName, Table.TableType.OLAP);
+            olapTable = (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/TableRowCountAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/TableRowCountAction.java
index 356aa8224f..c67983cf92 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
@@ -63,7 +63,7 @@ public class TableRowCountAction extends RestBaseController {
             OlapTable olapTable;
             try {
                 Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
-                olapTable = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
+                olapTable = (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 dc3b2c24dc..a2a8ff4fea 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
@@ -66,7 +66,7 @@ public class TableSchemaAction extends RestBaseController {
             OlapTable table;
             try {
                 Database db = Catalog.getCurrentCatalog().getDbOrMetaException(fullDbName);
-                table = db.getTableOrMetaException(tblName, Table.TableType.OLAP);
+                table = (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/load/Load.java b/fe/fe-core/src/main/java/org/apache/doris/load/Load.java
index 91eee1c6c8..53426529c9 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
@@ -2452,7 +2452,7 @@ public class Load {
 
     public void replayClearRollupInfo(ReplicaPersistInfo info, Catalog catalog) throws MetaNotFoundException {
         Database db = catalog.getDbOrMetaException(info.getDbId());
-        OlapTable olapTable = db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
         olapTable.writeLock();
         try {
             Partition partition = olapTable.getPartition(info.getPartitionId());
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 d8a8ec210c..eb956f43dd 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
@@ -787,7 +787,8 @@ public abstract class RoutineLoadJob extends AbstractTxnStateChangeCallback impl
 
     private void initPlanner() throws UserException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        planner = new StreamLoadPlanner(db, db.getTableOrMetaException(this.tableId, Table.TableType.OLAP), this);
+        planner = new StreamLoadPlanner(db,
+            (OlapTable) db.getTableOrMetaException(this.tableId, Table.TableType.OLAP), this);
     }
 
     public TExecPlanFragmentParams plan(TUniqueId loadId, long txnId) throws UserException {
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 c21691aff1..cae82d0c24 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
@@ -508,7 +508,7 @@ public class RoutineLoadManager implements Writable {
     private Set<Tag> getTagsFromReplicaAllocation(long dbId, long tblId) throws LoadException {
         try {
             Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-            OlapTable tbl = db.getTableOrMetaException(tblId, Table.TableType.OLAP);
+            OlapTable tbl = (OlapTable) db.getTableOrMetaException(tblId, Table.TableType.OLAP);
             tbl.readLock();
             try {
                 PartitionInfo partitionInfo = tbl.getPartitionInfo();
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 79ea3c2c23..3649b2bae8 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
@@ -892,7 +892,7 @@ public class ReportHandler extends Daemon {
         long rowCount = backendTabletInfo.getRowCount();
 
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(dbId);
-        OlapTable olapTable = db.getTableOrMetaException(tableId, Table.TableType.OLAP);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, Table.TableType.OLAP);
         olapTable.writeLockOrMetaException();
         try {
             Partition partition = olapTable.getPartition(partitionId);
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 65f4a045bf..3eb7f1abf7 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
@@ -114,7 +114,7 @@ public class AlterJobV2Test {
             Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
 
             Database db = Catalog.getCurrentCatalog().getDbOrMetaException(alterJobV2.getDbId());
-            OlapTable tbl = db.getTableOrMetaException(alterJobV2.getTableId(), Table.TableType.OLAP);
+            OlapTable tbl = (OlapTable) db.getTableOrMetaException(alterJobV2.getTableId(), Table.TableType.OLAP);
             while (tbl.getState() != OlapTable.OlapTableState.NORMAL) {
                 Thread.sleep(1000);
             }
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 293d73341f..767f65a41c 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
@@ -1014,7 +1014,7 @@ public class AlterTest {
         createTable(createOlapTblStmt);
 
         Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test");
-        MysqlTable mysqlTable = db.getTableOrMetaException("mysql_table", Table.TableType.MYSQL);
+        MysqlTable mysqlTable = (MysqlTable) db.getTableOrMetaException("mysql_table", Table.TableType.MYSQL);
 
         String alterEngineStmt = "alter table test.mysql_table modify engine to odbc";
         alterTable(alterEngineStmt, true);
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 54b286d77f..144ee765cc 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
@@ -81,7 +81,7 @@ public class DropPartitionTest {
     @Test
     public void testNormalDropPartition() throws Exception {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
-        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable table = (OlapTable) 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;";
@@ -101,7 +101,7 @@ public class DropPartitionTest {
     @Test
     public void testForceDropPartition() throws Exception {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
-        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable table = (OlapTable) 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;";
@@ -120,7 +120,7 @@ public class DropPartitionTest {
     @Test
     public void testDropPartitionAndReserveTablets() throws Exception {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test");
-        OlapTable table = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable table = (OlapTable) 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/utframe/AnotherDemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
index 788dd109d6..5629e695f9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/AnotherDemoTest.java
@@ -104,7 +104,7 @@ public class AnotherDemoTest {
         Catalog.getCurrentCatalog().createTable(createTableStmt);
         // 4. get and test the created db and table
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:db1");
-        OlapTable tbl = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         tbl.readLock();
         try {
             Assert.assertNotNull(tbl);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java
index ba9a6a82e3..9f3e3453b3 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoTest.java
@@ -66,7 +66,7 @@ public class DemoTest extends TestWithFeService {
 
         // 4. get and test the created db and table
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:db1");
-        OlapTable tbl = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         tbl.readLock();
         try {
             Assertions.assertNotNull(tbl);
@@ -94,7 +94,7 @@ public class DemoTest extends TestWithFeService {
             Assertions.assertEquals(JobState.FINISHED, alterJobV2.getJobState());
         }
 
-        OlapTable tbl1 = db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
+        OlapTable tbl1 = (OlapTable) db.getTableOrMetaException("tbl1", Table.TableType.OLAP);
         tbl1.readLock();
         try {
             Assertions.assertEquals(2, tbl1.getBaseSchema().size());


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