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 933855256b13dd..fe3537af0c017f 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.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 void processCreateMaterializedView(CreateMaterializedViewStmt stmt) 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 alterClauses, @@ -260,17 +251,11 @@ private void processModifyColumnComment(Database db, OlapTable tbl, List 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 void processAlterTable(AlterTableStmt stmt) throws UserException { 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 @@ private void processReplaceTable(Database db, OlapTable origTable, List 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 void modifyPartitionsProperty(Database db, 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 61fdcca1f7ba9f..2f825b27bba876 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 @@ protected void jobDone(AlterJob alterJob) { } } - 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 Integer getAlterJobNumByState(JobState state) { * 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 15d5a39da85eb2..6a31be1b39ade5 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 synchronized final boolean cancel(String errMsg) { * 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 @@ protected boolean checkTableStable(Database db) throws AlterCancelException { 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 7fab577bb680d9..f1d231a7db962c 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.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 @@ private long dropMaterializedView(String mvName, OlapTable olapTable) { 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 @@ private void removeJobFromRunningQueue(AlterJobV2 alterJob) { } 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 void replayAlterJobV2(AlterJobV2 alterJob) { 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 @@ private void runOldAlterJob() { // 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 @@ private void getOldAlterJobInfos(Database db, List> rollupJobIn 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 void cancel(CancelStmt stmt) throws DdlException { 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 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 8a7830a2e4fffb..7195d58b8ccd6c 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 int checkOrResendClearTasks() { 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 int checkOrResendClearTasks() { 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 boolean sendTasks() { // 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 boolean sendTasks() { 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 int tryFinishJob() { 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 synchronized void clear() { @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 void replayInitJob(Database db) { @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 entry : this.partitionIdToRollupIndex.entrySet()) { @@ -908,7 +920,13 @@ public void replayFinish(Database db) { 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 void replayFinish(Database db) { @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 void replayCancel(Database db) { @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 void getJobInfo(List> jobInfos, OlapTable tbl) { // 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 dbd020476fbf5e..ad410b7dad3eb8 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 @@ protected void runPendingJob() throws AlterCancelException { 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 @@ protected void runPendingJob() throws AlterCancelException { } } MarkedCountDownLatch countDownLatch = new MarkedCountDownLatch(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 @@ private void addRollupIndexToCatalog(OlapTable tbl) { } 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 @@ protected void runWaitingTxnJob() throws AlterCancelException { } 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 @@ protected void runWaitingTxnJob() throws AlterCancelException { @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 @@ private void cancelInternal() { 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 @@ protected boolean isPreviousLoadFinished() throws AnalysisException { * 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 @@ private void addTabletToInvertedIndex(OlapTable tbl) { * 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 @@ private void replayPendingJob(RollupJobV2 replayedJob) { * 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 @@ private void replayCancelled(RollupJobV2 replayedJob) { @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 List> getUnfinishedTasks(int limit) { if (jobState == JobState.RUNNING) { List tasks = rollupBatchTask.getUnfinishedTasks(limit); for (AgentTask agentTask : tasks) { - AlterReplicaTask rollupTask = (AlterReplicaTask)agentTask; + AlterReplicaTask rollupTask = (AlterReplicaTask) agentTask; List info = Lists.newArrayList(); info.add(String.valueOf(rollupTask.getBackendId())); info.add(String.valueOf(rollupTask.getBaseTabletId())); @@ -812,10 +787,14 @@ public void gsonPostProcess() throws IOException { 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 e25a27fd71c777..b54847ee40dfda 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 @@ private void runOldAlterJob() { // 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 @@ private void getOldAlterJobInfos(Database db, List> schemaChang } 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 @@ private void sendClearAlterTask(Database db, OlapTable olapTable) { */ public void updateTableInMemoryMeta(Database db, String tableName, Map properties) throws UserException { List 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 void updatePartitionsInMemoryMeta(Database db, String tableName, List partitionNames, Map 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 void updatePartitionInMemoryMeta(Database db, boolean isInMemory) throws UserException { // be id -> Map>> 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 void cancel(CancelStmt stmt) throws DdlException { 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 feb442b042a3e2..9bde44b74f339f 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 synchronized int getFinishedReplicaNumByIndexId(long indexId) { } 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 int checkOrResendClearTasks() { 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 int checkOrResendClearTasks() { } 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 boolean sendTasks() { 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 boolean sendTasks() { 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 void handleFinishedReplica(AgentTask task, TTabletInfo finishTabletInfo, 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 int tryFinishJob() { 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 int tryFinishJob() { @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 synchronized void clear() { @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 void replayInitJob(Database db) { @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 void replayFinish(Database db) { 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 void replayFinish(Database db) { @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 3292d86ff2e08f..b5bd9a7e99c250 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 @@ private void pruneMeta() { 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 @@ protected void runPendingJob() throws AlterCancelException { } MarkedCountDownLatch 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 @@ protected void runWaitingTxnJob() throws AlterCancelException { } 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 @@ protected void runRunningJob() throws AlterCancelException { // 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 @@ private void cancelInternal() { 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 @@ protected boolean isPreviousLoadFinished() throws AnalysisException { * 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 cell : partitionIndexMap.cellSet()) { @@ -686,7 +667,7 @@ private void replayCreateJob(SchemaChangeJobV2 replayedJob) { 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 @@ private void replayCreateJob(SchemaChangeJobV2 replayedJob) { } // 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 @@ private void replayCreateJob(SchemaChangeJobV2 replayedJob) { * 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 @@ private void replayPendingJob(SchemaChangeJobV2 replayedJob) { * 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 @@ private void replayCancelled(SchemaChangeJobV2 replayedJob) { @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 a388703b839e41..d9c7aca57cd0b0 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 TableName getTbl() { } @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 void analyze(Analyzer analyzer) throws AnalysisException, UserException { 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 2a30367b7167aa..3c5f7e39439df4 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 TableRef resolveTableRef(TableRef tableRef) throws AnalysisException { 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 TableRef resolveTableRef(TableRef tableRef) throws AnalysisException { } } - 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 a2435fe7c6cead..656f4454e209c6 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.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 void analyze(Analyzer analyzer) throws UserException { 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 292eb84e653e61..ec9d58afbda684 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 void checkDBTable(Analyzer analyzer) throws AnalysisException { 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 9f5227571a284e..b617672486162f 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.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 @@ private void analyzeColumnToHadoopFunction(String columnName, Expr child1) throw } 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 @@ private void analyzeSequenceCol(String fullDbName) throws AnalysisException { } } 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 e4ab7318a216dd..8d7cccfef29e3d 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 void analyze(Analyzer analyzer) throws AnalysisException, UserException { 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 4e2f8816304f02..00f1881357ec47 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 @@ private void analyzeEncryptKey(Analyzer analyzer) throws AnalysisException { 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 0810b44be6730a..7b1eec34522b01 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 void analyze(Analyzer analyzer) throws UserException { } 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 5f09a1454d0a5b..83fab795a81037 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 @@ private void checkFromHiveTable(Analyzer analyzer) throws AnalysisException { 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 a624cb0d0c0b3f..c93efd848041e5 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 void analyzeImpl(Analyzer analyzer) throws AnalysisException { 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 64e3525898b7ab..bc1238378acaef 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 void getTables(Analyzer analyzer, Map tableMap, Set 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 void analyze(Analyzer analyzer) throws UserException { // 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 void analyze(Analyzer analyzer) throws UserException { 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 368d6c7c8a0de9..f68d58c711a1fe 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.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 void analyze(Analyzer analyzer) throws UserException { 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 146a57a119d3b4..471e30657ea246 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 void getTables(Analyzer analyzer, Map tableMap, Set 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 ebed55571309f3..a50f396fcb9ed8 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 void analyzeSyntax(Analyzer analyzer) throws AnalysisException, UserExcep } - 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 1797db78e79320..f02787d727d18b 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 void analyze(Analyzer analyzer) throws UserException { 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 int compare(Table t1, Table t2) { 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 954fe5c5e277af..1576b94fc05298 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 void analyze(Analyzer analyzer) throws UserException { 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 d43f0086cf6f7b..4ead82b910e761 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 @@ 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 void analyze(Analyzer analyzer) throws AnalysisException, UserException { 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 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 4853b3fb73884b..63fb7d779519e8 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.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 @@ private void analyzeTargetTable(Analyzer analyzer) throws AnalysisException { 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 5b18f2fee31389..0571059c28b3f2 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 void process(AbstractBackupStmt stmt) throws DdlException { // 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 @@ private void backup(Repository repository, Database db, BackupStmt stmt) throws // 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 void checkAndFilterRestoreOlapTableExistInSnapshot(Map 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 f368ca48f24a05..ab902d57329eea 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.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 @@ private void checkIfNeedCancel() { 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 @@ private void checkIfNeedCancel() { * 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 @@ private void checkAndPrepareMeta() { // 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 @@ private void checkAndPrepareMeta() { } 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 @@ private void checkAndPrepareMeta() { } } 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 @@ private void checkAndPrepareMeta() { 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 @@ private void checkAndPrepareMeta() { // 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 @@ private void checkAndPrepareMeta() { // 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 @@ private void checkAndPrepareMeta() { // generate create replica tasks for all restored partitions for (Pair 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 @@ private void checkAndPrepareMeta() { } 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 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 @@ private void prepareAndSendSnapshotTaskForOlapTable(Database db) { 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 @@ private boolean downloadAndDeserializeMetaInfo() { } 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 @@ private void replayCheckAndPrepareMeta() { // restored partitions for (Pair 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 @@ private void replayCheckAndPrepareMeta() { } } } - + // restored tables for (Table restoreTbl : restoredTbls) { @@ -1167,7 +1174,7 @@ private void downloadSnapshots() { for (long dbId : dbToSnapshotInfos.keySet()) { List 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 @@ private void downloadSnapshots() { 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 @@ private void waitingAllTabletsCommitted() { } 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 @@ private Status allTabletCommitted(boolean isReplay) { // 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 @@ private Status allTabletCommitted(boolean isReplay) { catalog.getEditLog().logRestoreJob(this); } - + LOG.info("job is finished. is replay: {}. {}", isReplay, this); return Status.OK; } @@ -1530,7 +1537,7 @@ public void cancelInternal(boolean isReplay) { } // 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 void cancelInternal(boolean isReplay) { // remove restored partitions for (Pair 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 void cancelInternal(boolean isReplay) { 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 35a881acd89b57..a5d102a208e935 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.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.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 @@ private Catalog(boolean isCheckpointCatalog) { 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 @@ private Catalog(boolean isCheckpointCatalog) { 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 @@ private boolean tryLock(boolean mustLock) { LOG.debug("catalog lock is held by: {}", Util.dumpThread(owner, 10)); } } - + if (mustLock) { continue; } else { @@ -816,7 +817,7 @@ public void waitForReady() throws InterruptedException { 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 long loadFrontends(DataInputStream dis, long checksum) throws IOException Frontend fe = Frontend.read(dis); replayAddFrontend(fe); } - + size = dis.readInt(); newChecksum ^= size; for (int i = 0; i < size; i++) { @@ -1802,7 +1803,7 @@ public long loadAlterJob(DataInputStream dis, long checksum, JobType type) throw 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 long saveLoadJobsV2(CountingDataOutputStream dos, long checksum) throws I 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 void dropDb(DropDbStmt stmt) throws DdlException { 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 void replayDropDb(String dbName, boolean isForceDrop) throws DdlException 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 void recoverDatabase(RecoverDbStmt recoverStmt) throws DdlException { 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 void recoverTable(RecoverTableStmt recoverStmt) throws DdlException { 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 void replayRecoverDatabase(RecoverInfo info) { 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 void alterDatabaseQuota(AlterDatabaseQuotaStmt stmt) throws DdlException 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 void replayRenameDatabase(String dbName, String newDbName) { * 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 void createTable(CreateTableStmt stmt) throws DdlException { } // 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 void createTable(CreateTableStmt stmt) throws DdlException { 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 void addPartition(Database db, String tableName, AddPartitionClause addPa DistributionDesc distributionDesc = addPartitionClause.getDistributionDesc(); boolean isTempPartition = addPartitionClause.isTempPartition(); - DistributionInfo distributionInfo = null; - OlapTable olapTable = null; - + DistributionInfo distributionInfo; Map indexIdToMeta; - Set bfColumns = null; - + Set 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 void addPartition(Database db, String tableName, AddPartitionClause addPa 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 void addPartition(Database db, String tableName, AddPartitionClause addPa } } - 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 void dropPartition(Database db, OlapTable olapTable, DropPartitionClause 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 void replayErasePartition(long partitionId) { 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 @@ private void createOlapTable(Database db, CreateTableStmt stmt) throws DdlExcept 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 void replayCreateTable(String dbName, Table table) { } } - public void replayAlterExternalTableSchema(String dbName, String tableName, List newSchema) throws DdlException { - Database db = this.fullNameToDb.get(dbName); - Table table = db.getTable(tableName); + public void replayAlterExternalTableSchema(String dbName, String tableName, List newSchema) throws MetaNotFoundException { + Database db = this.getDbOrMetaException(dbName); + Table table = db.getTableOrMetaException(tableName); table.writeLock(); try { table.setNewFullSchema(newSchema); @@ -4431,7 +4377,6 @@ private void createTablets(String clusterName, MaterializedIndex index, ReplicaS 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 @@ private void createTablets(String clusterName, MaterializedIndex index, ReplicaS // get backends from existing backend sequence chosenBackendIds = backendsPerBucketSeq.get(i); } - + // create replicas for (long backendId : chosenBackendIds) { long replicaId = getNextId(); @@ -4474,7 +4419,7 @@ private void createTablets(String clusterName, MaterializedIndex index, ReplicaS 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 void dropTable(DropTableStmt stmt) throws DdlException { 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 void dropTable(DropTableStmt stmt) throws DdlException { 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 void dropTable(DropTableStmt stmt) throws DdlException { 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 boolean unprotectDropTable(Database db, Table table, boolean isForceDrop, 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 void replayDropTable(Database db, long tableId, boolean isForceDrop) { } } - 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 void replayRecoverTable(RecoverInfo info) { } } - 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 @@ private void unprotectAddReplica(ReplicaPersistInfo info) { 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 @@ private void unprotectUpdateReplica(ReplicaPersistInfo info) { 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 String getToken() { 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 getDb(String dbName) { + return Optional.ofNullable(getDbNullable(dbName)); + } + + public Optional getDb(long dbId) { + return Optional.ofNullable(getDbNullable(dbId)); + } + + public Database + getDbOrException(String dbName, java.util.function.Function e) throws E { + Database db = getDbNullable(dbName); + if (db == null) { + throw e.apply(dbName); + } + return db; + } + + public Database + getDbOrException(long dbId, java.util.function.Function 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 getDbNames() { @@ -4848,7 +4799,7 @@ public HashMap getPartitionIdToStorageMediumMap() { List 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 HashMap getPartitionIdToStorageMediumMap() { // 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 HashMap getPartitionIdToStorageMediumMap() { Multimap 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 SqlBlockRuleMgr getSqlBlockRuleMgr() { return sqlBlockRuleMgr; } - public RoutineLoadTaskScheduler getRoutineLoadTaskScheduler(){ + public RoutineLoadTaskScheduler getRoutineLoadTaskScheduler() { return routineLoadTaskScheduler; } @@ -5258,7 +5209,7 @@ public void renameTable(Database db, Table table, TableRenameClause tableRenameC 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 void renameTable(Database db, Table table, TableRenameClause tableRenameC } // 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 void refreshExternalTableSchema(Database db, Table table, List ne 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 void modifyTableColocate(Database db, OlapTable table, String colocateGro } 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 void modifyTableColocate(Database db, OlapTable table, String colocateGro // set to an already exist colocate group, check if this table can be added to this group. groupSchema.checkColocateSchema(table); } - + List> backendsPerBucketSeq = null; if (groupSchema == null) { // assign to a newly created group, set backends sequence. @@ -5403,26 +5354,22 @@ public void modifyTableColocate(Database db, OlapTable table, String colocateGro 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 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 void renameRollup(Database db, OlapTable table, RollupRenameClause rename } } - 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 indexNameToIdMap = table.getIndexNameToId(); + String rollupName = olapTable.getIndexNameById(indexId); + Map 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 void renamePartition(Database db, OlapTable table, PartitionRenameClause } } - 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 void modifyTableDynamicPartition(Database db, OlapTable table, Map 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 void modifyTableInMemoryMeta(Database db, OlapTable table, Map 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 void replayModifyTableProperty(short opCode, ModifyTablePropertyOperation // 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 void modifyDefaultDistributionBucketNum(Database db, OlapTable olapTable, } } - 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 void changeDb(ConnectContext ctx, String qualifiedDb) throws DdlException 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 void createView(CreateViewStmt stmt) throws DdlException { 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 void createView(CreateViewStmt stmt) throws DdlException { } 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 void createCluster(CreateClusterStmt stmt) throws DdlException { } private void unprotectCreateCluster(Cluster cluster) { - final Iterator 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 Set getMigrations() { final Set 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 long loadCluster(DataInputStream dis, long checksum) throws IOException, // 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 String dumpImage() { 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 void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti // check, and save some info which need to be checked again later Map 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 void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti 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 newPartitions = Lists.newArrayList(); // tabletIdSet to save all newly created tablet ids. @@ -6745,11 +6654,8 @@ public void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti // 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 void truncateTable(TruncateTableStmt truncateTableStmt) throws DdlExcepti 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 @@ private void truncateTableInternal(OlapTable olapTable, List newParti } } - 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 void replayTruncateTable(TruncateTableInfo info) { 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 void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { // but we need to get replica from db->tbl->partition->... List 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 void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { info.getReplicaId(), info.getTabletId(), info.getBackendId()); } } finally { - tbl.writeUnlock(); + olapTable.writeUnlock(); } } } @@ -6983,20 +6870,15 @@ public void convertDistributionType(Database db, OlapTable tbl) throws DdlExcept } } - 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 void replaceTempPartition(Database db, OlapTable olapTable, ReplacePartit 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 void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP 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 long loadPlugins(DataInputStream dis, long checksum) throws IOException { 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 void uninstallPlugin(UninstallPluginStmt stmt) throws IOException, UserEx 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 void setReplicaStatus(AdminSetReplicaStatusStmt stmt) throws MetaNotFound 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 bdee43118b9d21..333a89e7678500 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 void addTabletToInvertedIndex() { // 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 void addTabletToInvertedIndex() { 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 756d268cf368bd..68c552d3750c31 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 GroupId changeGroup(long dbId, OlapTable tbl, String oldGroup, String new } } - 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 @@ private void convertedToNewMembers(Multimap tmpGroup2Tables, Map 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 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 c3ad35a91c5bc5..7c4db1e1675e24 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.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.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.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 void dropTable(String tableName) { 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 Set getTableNamesWithLock() { /** * 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 Table getTable(String tableName) { } /** - * 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 getTable(String tableName) { + return Optional.ofNullable(getTableNullable(tableName)); + } + + public Optional
getTable(long tableId) { + return Optional.ofNullable(getTableNullable(tableId)); + } + + public Table getTableOrException(String tableName, java.util.function.Function 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 Table getTableOrException(long tableId, java.util.function.Function 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 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 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 b555f98a06657e..cede16e3197817 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 @@ 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 78666f64e586d8..b17c2650e3eb97 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 @@ private void initTables() { } @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 feecd2ab90ebc4..873426449c66a6 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.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 @@ private static List> getTabletStatus(String dbName, String tblName, 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 @@ private static List> getTabletStatus(String dbName, String tblName, } } } finally { - tbl.readUnlock(); + olapTable.readUnlock(); } return result; @@ -177,19 +167,10 @@ private static List> getTabletDistribution(String dbName, String tb 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 partitionIds = Lists.newArrayList(); if (partitionNames == null) { for (Partition partition : olapTable.getPartitions()) { @@ -251,7 +232,7 @@ private static List> getTabletDistribution(String dbName, String tb } } finally { - tbl.readUnlock(); + olapTable.readUnlock(); } return result; @@ -282,19 +263,11 @@ private static List> getDataSkew(String dbName, String tblName, Par 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 @@ private static List> getDataSkew(String dbName, String tblName, Par 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 898a97a9088738..ed02c39f7e0292 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 @@ protected void runAfterCatalogReady() { start = System.currentTimeMillis(); List 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 d566eb0d7e1aad..00bc1a141a925a 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 @@ private void relocateAndBalanceGroup() { // get all groups Set 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 @@ private void matchGroup() { Set groupIds = colocateIndex.getAllGroupIds(); for (GroupId groupId : groupIds) { List tableIds = colocateIndex.getAllTableIds(groupId); - Database db = catalog.getDb(groupId.dbId); + Database db = catalog.getDbNullable(groupId.dbId); if (db == null) { continue; } @@ -196,7 +196,7 @@ private void matchGroup() { 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 0162bc7f7cb311..e75624feb13e17 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 @@ private void executeDynamicPartition(Collection> dynamicPartiti Pair 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 @@ private void executeDynamicPartition(Collection> dynamicPartiti 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 @@ private void clearDropPartitionFailedMsg(long tableId) { 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 77238c43b50fa1..f6bd08d57b616b 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.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 @@ private void checkTablets() { OUT: for (long dbId : copiedPrios.rowKeySet()) { - Database db = catalog.getDb(dbId); + Database db = catalog.getDbNullable(dbId); if (db == null) { continue; } List aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); Map> 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 @@ private void checkTablets() { List 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 @@ private void removePriosIfNecessary() { while (iter.hasNext()) { Map.Entry>> 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 @@ private void removePriosIfNecessary() { while (jter.hasNext()) { Map.Entry> 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 List> getPriosInfo() { public static RepairTabletInfo getRepairTabletInfo(String dbName, String tblName, List 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 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 static RepairTabletInfo getRepairTabletInfo(String dbName, String tblName } } } 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 8e3ab510660b23..d1ffbc1dcc0160 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 void releaseResource(TabletScheduler tabletScheduler, boolean reserveTabl 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 void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) } // 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 void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) 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 aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true); short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId); Pair pair = tablet.getHealthStatusWithPriority( @@ -835,9 +829,9 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) 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 void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) 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 9c4b2b4c5b822d..56aee101b7b884 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 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) 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 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 9fbd6b78afe84b..14b5ed9cbee36d 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 MetaNotFoundException(InternalErrorCode errcode, String msg) { 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 9734a3189cf58d..08419d4f46a7bf 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 ProcNodeInterface lookup(String dbIdStr) throws AnalysisException { 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 ProcResult fetchResult() throws AnalysisException { // get info List> dbInfos = new ArrayList>(); 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 c9da8e3aaba871..ba1dc88ba13e9c 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 ProcNodeInterface lookup(String dbIdStr) throws AnalysisException { 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 ProcResult fetchResult() throws AnalysisException { result.setNames(TITLE_NAMES); List 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 596267cb4228ea..6f051bc300ba54 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 ProcResult fetchResult() throws AnalysisException { // 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 380618d3d3667f..58f079a63ccfe7 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 ProcNodeInterface lookup(String tableIdStr) throws AnalysisException { 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 3bafb68cad6bb6..7ee6354b501fdc 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 SmallFileMgr() { 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 @@ private File getAbsoluteFile(long dbId, String catalog, String fileName) { } public List> 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> 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 aa183b18cc327c..a10e7e4e8f0632 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 boolean sendTasks() { 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 boolean sendTasks() { 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 synchronized int tryFinishJob() { 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 052847b4e6382e..acab74d9b9012c 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.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 @@ private List chooseTablets() { // skip 'information_schema' database continue; } - Database db = catalog.getDb(dbId); + Database db = catalog.getDbNullable(dbId); if (db == null) { continue; } @@ -362,9 +363,9 @@ public void handleFinishedConsistencyCheck(CheckConsistencyTask task, long check 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 2d94ee8df10551..95816410c31c5a 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 void loadTableFromCatalog() { } List 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
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 ec0c37d924038d..23f12f441a196f 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 void executeWithoutPassword(BaseRequest request, BaseResponse response) t // 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 61c26319ad7cab..e9c23429fa2451 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 void executeWithoutPassword(BaseRequest request, BaseResponse response) 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 createTableStmt = Lists.newArrayList(); List addPartitionStmt = Lists.newArrayList(); List 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 57b23480c9aa24..6be12368eb263c 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 void executeWithoutPassword(BaseRequest request, BaseResponse response) // 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 50bbdd6d62086d..6f36c5736fe9c3 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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response 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> 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 c9b9f98ce8e643..e134fc3a73fe87 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.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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response Map 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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } // 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 faf99f386e89e2..e3bbef3c405724 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 Map getDataSize() { Map result = new HashMap(); List 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
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 Map getDataSize() { } // 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 b1906c51d636ac..f9968d2fafdcf6 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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response } 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
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 27042d707ba60b..7634fab5cef072 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.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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response 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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response // 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 b255625a8fddf9..3b755478baff5f 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.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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response 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 c9868706988779..e298042af9176b 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 @@ 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.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 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response 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 ff5f971b370f20..4b716b47b6d131 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 @@ 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 Object execute(@PathVariable(value = DB_KEY) final String dbName, 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 45ac72bdb6e475..ce6d93ac2066f2 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 @@ 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 Object execute(HttpServletRequest request, HttpServletResponse response) } 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 createTableStmt = Lists.newArrayList(); List addPartitionStmt = Lists.newArrayList(); List 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 194b63fafaeb13..208008a97d662e 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 @@ 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 Object execute(@PathVariable(value = DB_KEY) final String dbName, 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 aedc4690b87b26..e00d54ddc3bcd6 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.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 Object getTables( 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 tblNames = Lists.newArrayList(); @@ -215,19 +218,18 @@ public Object getTableSchema( 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> 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 bfe75633c50a3f..c4f7d163c59f95 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 @@ protected Object rowcount(HttpServletRequest request, HttpServletResponse respon String fullDbName = getFullDbName(dbName); Map 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 d98e2489bd660a..e08ebe5fd75d23 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 @@ private Map getDataSize() { Map result = new HashMap(); List 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 cf35a8e932e40b..3845a2ec2b9815 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.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 @@ protected Object check_storagetype(HttpServletRequest request, HttpServletRespon } 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> 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 c9a87d9248796a..d64a7880f9b0dd 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 Object query_plan( 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 534e7741ad1238..8d9d7f9d6b3a05 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 Object count( 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 a776fce96bb209..856abc1d6efa1f 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 @@ protected Object schema( 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 c3f64e25e2ac1a..43e98198f3f393 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.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 Object getTables( 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 tblNames = Lists.newArrayList(); @@ -207,30 +210,25 @@ public Object getTableSchema( 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 6fff16fa250c68..84436f65193d88 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.HttpServletResponse; import java.util.List; import java.util.Map; +import java.util.Objects; @RestController @RequestMapping("/rest/v2") @@ -60,13 +60,8 @@ public Object clusterOverview(HttpServletRequest request, HttpServletResponse re } private int getTblCount(Catalog catalog) { - int tblCount = 0; - List 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 bc03e6c24e307d..b19d312863fc17 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 BrokerFileGroup(DataDescription dataDescription) { // 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 void parse(Database db, DataDescription dataDescription) throws DdlExcept 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 void parse(Database db, DataDescription dataDescription) throws DdlExcept // 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 void parse(Database db, DataDescription dataDescription) throws DdlExcept 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 36cd63940a4cad..d23b6049e2ff91 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 void process(DeleteStmt stmt) throws DdlException, QueryStateException { List partitionNames = stmt.getPartitionNames(); boolean noPartitionSpecified = partitionNames.isEmpty(); List 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 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 void process(DeleteStmt stmt) throws DdlException, QueryStateException { 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 deleteConditions = Lists.newArrayList(); @@ -211,7 +200,7 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { 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 void process(DeleteStmt stmt) throws DdlException, QueryStateException { } throw new DdlException(t.getMessage(), t); } finally { - table.readUnlock(); + olapTable.readUnlock(); } long timeoutMs = deleteJob.getTimeoutMs(); @@ -344,14 +333,14 @@ public void process(DeleteStmt stmt) throws DdlException, QueryStateException { 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 @@ private void checkDeleteV2(OlapTable table, List partitions, List
> getDeleteInfosByDb(long dbId) {
         LinkedList> infos = new LinkedList>();
-        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 c94980b71dd28f..61dcad51255d63 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 DeleteJob(long id, long transactionId, String label,
      */
     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 30c0f453679124..fde11b99607680 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 ExportJob(long jobId) {
 
     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 void setJob(ExportStmt stmt) throws UserException {
 
         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 void setJob(ExportStmt stmt) throws UserException {
         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 51019ca17b0b86..13380029175d2e 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 List> getExportJobInfosByIdOrState(
                 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 bc78e44ba610da..f62d4c483c2dd5 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 boolean addMiniLoadJob(TMiniLoadRequest request) throws DdlException {
     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 @@ private void addLoadJob(LoadJob job, Database db) throws DdlException {
         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 @@ private LoadJob createLoadJob(LoadStmt stmt, EtlJobType etlJobType,
 
             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 static void checkAndCreateSource(Database db, DataDescription dataDescrip
         String tableName = dataDescription.getTableName();
         Map>> 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 static void checkAndCreateSource(Database db, DataDescription dataDescrip
             }
 
             // 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 void replayLoadingLoadJob(LoadJob job) throws DdlException {
     // 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 boolean registerMiniLabel(String fullDbName, String label, long timestamp
     }
 
     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 @@ private boolean checkMultiLabelUsed(long dbId, String label, long timestamp) thr
 
     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> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -1636,10 +1615,7 @@ public boolean cancelLoadJob(CancelLoadStmt stmt, boolean isAccurateMatch) throw
         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 loadJobs = Lists.newArrayList();
         readLock();
@@ -1724,11 +1700,8 @@ public boolean cancelLoadJob(CancelLoadStmt stmt) throws DdlException {
         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> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -2112,7 +2085,7 @@ public List> getLoadJobUnfinishedInfo(long jobId) {
         }
 
         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 List> getLoadJobUnfinishedInfo(long jobId) {
 
                 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 JobInfo(String dbName, String label, String clusterName) {
     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> labelToLoadJobs = dbLabelToLoadJobs.get(db.getId());
@@ -2330,7 +2300,7 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) {
             Map 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 void unprotectQuorumLoadJob(LoadJob job, Database db) {
             if (idToTableLoadInfo != null) {
                 for (Entry 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 entry : tableLoadInfo.getIdToPartitionLoadInfo().entrySet()) {
                         long partitionId = entry.getKey();
@@ -2403,9 +2376,9 @@ public void unprotectQuorumLoadJob(LoadJob job, Database db) {
         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 tableIds = Lists.newArrayList();
         long tblId = job.getTableId();
@@ -2415,13 +2388,7 @@ public void replayQuorumLoadJob(LoadJob job, Catalog catalog) throws DdlExceptio
             tableIds.addAll(job.getIdToTableLoadInfo().keySet());
         }
 
-        List
tables = null; - try { - tables = db.getTablesOnIdOrderOrThrowException(tableIds); - } catch (MetaNotFoundException e) { - LOG.error("should not happen", e); - return; - } + List
tables = db.getTablesOnIdOrderOrThrowException(tableIds); MetaLockUtils.writeLockTables(tables); try { @@ -2446,7 +2413,7 @@ public void unprotectFinishLoadJob(LoadJob job, Database db) { Map 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 void unprotectFinishLoadJob(LoadJob job, Database db) { 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 void replayFinishLoadJob(LoadJob job, Catalog catalog) { } } - 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 void clearJob(LoadJob job, JobState srcState) { } 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 boolean updateLoadJobState(LoadJob job, JobState destState) { public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType cancelType, String msg, List 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 @@ private boolean processQuorumFinished(LoadJob job, Database db) { Map idToTableLoadInfo = job.getIdToTableLoadInfo(); for (Entry 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 @@ private boolean processQuorumFinished(LoadJob job, Database db) { // update partition version and index row count for (Entry 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 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 0b8d847b97e12e..bf804e691cb04e 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 @@ private void runOneLoadingJob(LoadJob job) { 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 @@ private void tryCommitJob(LoadJob job, List
tables) { private Set submitPushTasks(LoadJob job, Database db) { Map tabletLoadInfos = job.getIdToTabletLoadInfo(); - boolean needDecompress = (job.getEtlJobType() == EtlJobType.HADOOP) ? true : false; + boolean needDecompress = job.getEtlJobType() == EtlJobType.HADOOP; AgentBatchTask batchTask = new AgentBatchTask(); Set jobTotalTablets = new HashSet(); Map idToTableLoadInfo = job.getIdToTableLoadInfo(); for (Entry 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 @@ private void runOneQuorumFinishedJob(LoadJob job) { // 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 fb713d68b8b9c5..a88c1c2a01e42f 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 @@ protected void runAfterCatalogReady() { } 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 93f60ac8be23f7..afd153de876725 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 @@ private void createLoadingTask(Database db, BrokerPendingTaskAttachment attachme FileGroupAggKey aggKey = entry.getKey(); List 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 77d7e6f01ff338..ba80021bf5a477 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.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 BulkLoadJob(EtlJobType jobType, long dbId, String label, OriginStatement 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 @@ private void checkAndSetDataSourceInfo(Database db, List dataDe } 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 getTableNamesForShow() { - Set 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 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 getTableNames() throws MetaNotFoundException { Set 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 void analyze() { 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 bc0db60531fc85..8c2bdce4f86318 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 InsertLoadJob(String label, long dbId, long tableId, long createTimestamp } 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 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 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 e33e98bf7e3277..c5c97b38b2bdbe 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 long getId() { } 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 @@ private void checkAuth(String command) throws DdlException { * @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 b8d8cabdf855bd..2e7e79ad049417 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 long createLoadJobFromMiniLoad(TMiniLoadBeginRequest request) throws User 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 void recordFinishedLoadJob(String label, String dbName, long tableId, Etl 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 void recordFinishedLoadJob(String label, String dbName, long tableId, Etl } 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 loadJobs = Lists.newArrayList(); @@ -355,10 +348,7 @@ public void cancelLoadJob(CancelLoadStmt stmt, boolean isAccurateMatch) throws D } 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 @@ private void analyzeLoadJobs() { } 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 b490dcaebfb5d5..a714475a69181f 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 LoadingTaskPlanner(Long loadJobId, long txnId, long dbId, OlapTable table 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 705e3f624200f8..65a9a852f0792b 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 Set getTableNames() throws MetaNotFoundException { } 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 85e1eae94655cf..41d850b295ecc8 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 void init() throws LoadException { } 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 tables = Maps.newHashMap(); Map> tableIdToPartitionIds = Maps.newHashMap(); Set allPartitionsTableIds = Sets.newHashSet(); prepareTablePartitionInfos(db, tableIdToPartitionIds, allPartitionsTableIds); - List
tableList = null; + List
tableList; try { tableList = db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(allPartitionsTableIds)); } catch (MetaNotFoundException e) { @@ -153,10 +150,7 @@ private void createEtlJobConf() throws LoadException { 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 @@ private void prepareTablePartitionInfos(Database db, Map> tableI 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 partitionIds = null; + Set partitionIds; if (tableIdToPartitionIds.containsKey(tableId)) { partitionIds = tableIdToPartitionIds.get(tableId); } else { @@ -486,10 +477,7 @@ private EtlFileGroup createEtlFileGroup(BrokerFileGroup fileGroup, Set tab Map 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 a3570556e60b1d..7a3b87b1f4b515 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.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 @@ private List getAllKafkaPartitions() throws UserException { 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 07854b7a7399be..f4d4d6ea615ef8 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 @@ 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 TRoutineLoadTask createRoutineLoadTask() throws UserException { 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 edca51188cbfed..734c8b3c7ec98a 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.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 void setOtherMsg(String otherMsg) { } 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 long getTableId() { } 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 void prepare() throws UserException { } 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 @@ private void executeTaskOnTxnStatusChanged(RoutineLoadTaskInfo routineLoadTaskIn } } - 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 @@ protected static void checkMeta(Database db, String tblName, RoutineLoadDesc rou } // check partitions - OlapTable olapTable = (OlapTable) table; olapTable.readLock(); try { for (String partName : partitionNames.getPartitionNames()) { @@ -1211,7 +1177,7 @@ private void executeCancel(ErrorReason reason) { 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 void update() throws UserException { } // 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 @@ protected abstract boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCo protected abstract String getStatistic(); public List getShowInfo() { - Database db = Catalog.getCurrentCatalog().getDb(dbId); - Table tbl = (db == null) ? null : db.getTable(tableId); + Optional database = Catalog.getCurrentCatalog().getDb(dbId); + Optional
table = database.flatMap(db -> db.getTable(tableId)); readLock(); try { @@ -1292,8 +1258,8 @@ public List getShowInfo() { 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 List> getTasksShowInfo() { } public String getShowCreateInfo() { - Database db = Catalog.getCurrentCatalog().getDb(dbId); - Table tbl = (db == null) ? null : db.getTable(tableId); + Optional database = Catalog.getCurrentCatalog().getDb(dbId); + Optional
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 @@ private static void appendProperties(StringBuilder sb, String key, Object value, } public List getShowStatistic() { - Database db = Catalog.getCurrentCatalog().getDb(dbId); + Optional database = Catalog.getCurrentCatalog().getDb(dbId); List 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 0f0a1230693101..e6a5263d3f582f 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 List checkPrivAndGetAllJobs(String dbName) throws MetaNotFoundException, DdlException, AnalysisException { List 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> jobMap = dbToNameToRoutineLoadJob.get(dbId); if (jobMap == null) { @@ -499,12 +496,8 @@ public List getJob(String dbFullName, String jobName, boolean in 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 1077734a40bf01..3ce5b48b0d0e58 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 enum JobState { 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 60a601965e85ea..8b06f0648b41bf 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 void pauseSyncJob(PauseSyncJobStmt stmt) throws DdlException { 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 syncJobs = Lists.newArrayList(); readLock(); @@ -101,7 +98,7 @@ public void pauseSyncJob(PauseSyncJobStmt stmt) throws DdlException { throw new DdlException("Load job does not exist"); } - List runningSyncJob = matchJobs.stream().filter(entity -> entity.isRunning()) + List 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 void resumeSyncJob(ResumeSyncJobStmt stmt) throws DdlException { 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 syncJobs = Lists.newArrayList(); readLock(); @@ -135,7 +129,7 @@ public void resumeSyncJob(ResumeSyncJobStmt stmt) throws DdlException { throw new DdlException("Load job does not exist"); } - List pausedSyncJob = matchJobs.stream().filter(entity -> entity.isPaused()) + List 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 void stopSyncJob(StopSyncJobStmt stmt) throws DdlException { 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 syncJobs = Lists.newArrayList(); @@ -237,10 +228,7 @@ public List getSyncJobs(SyncJob.JobState state) { } 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 1cb3c77a859f08..dce10bab55d630 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.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 void initChannels() throws DdlException { 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 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 1e8988d7b91319..b31f54494ac904 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 @@ private void commitAndPublishTxn() throws UserException { 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 static UpdateStmtExecutor fromUpdateStmt(UpdateStmt updateStmt) throws An 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 2b3217f07a52fe..55de34813ba155 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 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { 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 @@ private void finishRealtimePush(AgentTask task, TFinishTaskRequest request) { } 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 @@ private void finishPush(AgentTask task, TFinishTaskRequest request) { 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 @@ private void finishPush(AgentTask task, TFinishTaskRequest request) { 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 2181ff51664049..8a79f3dcb8e97d 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 @@ private void updatePartitionInMemoryInfo() { ImmutableSet.Builder builder = ImmutableSet.builder(); List 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 fe947324092524..867c634bbdc131 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 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap tabletDeleteFromMeta 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 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta } 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 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM 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 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM } 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 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon 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 c0f562b189cbb6..cc84395f1d6e7d 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.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 static void loadJournal(Catalog catalog, JournalEntity journal) { } 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 static void loadJournal(Catalog catalog, JournalEntity journal) { 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 6a0039576a2981..36b464120975d9 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 String getDatabase() { 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 056b8902a49ec4..a788efc0b22467 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 @@ private void handleFieldList() throws IOException { 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 bc3ba5e5439142..06d48599bca823 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 @@ private void handleShowEngines() { // 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 functions = showStmt.getIsBuiltin() ? ctx.getCatalog().getBuiltinFunctions() : db.getFunctions(); @@ -426,10 +423,7 @@ private void handleShowFunctions() throws AnalysisException { // 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> resultRowSet = Lists.newArrayList(); @@ -443,10 +437,7 @@ private void handleShowCreateFunction() throws AnalysisException { // 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 encryptKeys = db.getEncryptKeys(); List> rowSet = Lists.newArrayList(); @@ -541,7 +532,7 @@ private void handleShowDbId() throws AnalysisException { long dbId = showStmt.getDbId(); List> rows = Lists.newArrayList(); Catalog catalog = ctx.getCatalog(); - Database database = catalog.getDb(dbId); + Database database = catalog.getDbNullable(dbId); if (database != null) { List row = new ArrayList<>(); row.add(database.getFullName()); @@ -556,12 +547,13 @@ private void handleShowTableId() throws AnalysisException { List> rows = Lists.newArrayList(); Catalog catalog = ctx.getCatalog(); List 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 row = new ArrayList<>(); row.add(database.getFullName()); @@ -580,8 +572,9 @@ private void handleShowPartitionId() throws AnalysisException { List> rows = Lists.newArrayList(); Catalog catalog = ctx.getCatalog(); List 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 @@ private void handleShowDb() throws AnalysisException { private void handleShowTable() throws AnalysisException { ShowTableStmt showTableStmt = (ShowTableStmt) stmt; List> rows = Lists.newArrayList(); - Database db = ctx.getCatalog().getDb(showTableStmt.getDb()); - if (db != null) { - Map 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 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 entry : tableMap.entrySet()) { - if (showTableStmt.isVerbose()) { - rows.add(Lists.newArrayList(entry.getKey(), entry.getValue())); - } else { - rows.add(Lists.newArrayList(entry.getKey())); - } + for (Map.Entry 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 @@ private void handleShowTable() throws AnalysisException { private void handleShowTableStatus() throws AnalysisException { ShowTableStatusStmt showStmt = (ShowTableStatusStmt) stmt; List> 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 @@ private void handleShowVariables() throws AnalysisException { private void handleShowCreateDb() throws AnalysisException { ShowCreateDbStmt showStmt = (ShowCreateDbStmt) stmt; List> 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 @@ private void handleShowCreateDb() throws AnalysisException { // 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> 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 @@ private void handleDescribe() throws AnalysisException { private void handleShowColumn() throws AnalysisException { ShowColumnStmt showStmt = (ShowColumnStmt) stmt; List> 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 columns = table.getBaseSchema(); + for (Column col : columns) { + if (matcher != null && !matcher.match(col.getName())) { + continue; } - table.readLock(); - try { - List 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 @@ private void handleShowColumn() throws AnalysisException { private void handleShowIndex() throws AnalysisException { ShowIndexStmt showStmt = (ShowIndexStmt) stmt; List> 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 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 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 @@ private void handleShowLoad() throws AnalysisException { 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 of load(v1) and loadManager(v2) @@ -1066,10 +1027,7 @@ private void handleShowStreamLoad() throws AnalysisException { 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> streamLoadRecords = catalog.getStreamLoadRecordMgr().getStreamLoadRecordByDb(dbId, showStmt.getLabelValue(), showStmt.isAccurateMatch(), showStmt.getState()); @@ -1123,10 +1081,7 @@ private void handleShowLoadWarnings() throws AnalysisException { } 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 @@ private void handleShowDelete() throws AnalysisException { 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 @@ private void handleShowTablet() throws AnalysisException { // 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 @@ private void handleShowTablet() throws AnalysisException { 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 @@ private void handleShowTablet() throws AnalysisException { 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 @@ private void handleShowTablet() throws AnalysisException { rows.add(oneTablet); } } finally { - table.readUnlock(); + olapTable.readUnlock(); } } @@ -1657,10 +1598,7 @@ private void handleShowResources() { 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 @@ private void handleShowSnapshot() throws AnalysisException { 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 jobs = Catalog.getCurrentCatalog().getBackupHandler().getJobs(db.getId(), showStmt.getLabelPredicate()); @@ -1735,10 +1670,7 @@ private void handleShowBackup() throws AnalysisException { 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 jobs = Catalog.getCurrentCatalog().getBackupHandler().getJobs(db.getId(), showStmt.getLabelPredicate()); @@ -1753,10 +1685,7 @@ private void handleShowRestore() throws AnalysisException { 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> syncInfos = catalog.getSyncJobManager().getSyncJobsInfoByDbId(db.getId()); Collections.sort(syncInfos, new ListComparator>(0)); @@ -1858,7 +1787,7 @@ private void handleShowSmallFiles() throws AnalysisException { private void handleShowDynamicPartition() { ShowDynamicPartitionStmt showDynamicPartitionStmt = (ShowDynamicPartitionStmt) stmt; List> rows = Lists.newArrayList(); - Database db = ctx.getCatalog().getDb(showDynamicPartitionStmt.getDb()); + Database db = ctx.getCatalog().getDbNullable(showDynamicPartitionStmt.getDb()); if (db != null) { List
tableList = db.getTables(); for (Table tbl : tableList) { @@ -1912,10 +1841,7 @@ private void handleShowDynamicPartition() { // 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 75d1c1e6e79a05..742672c56aacb8 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 @@ private void beginTxn(String dbName, String tblName) throws UserException, TExce 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 a25f29fc14a5e0..81955dec1b6e16 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 TGetTablesResult getTableNames(TGetTablesParams params) throws TException // 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 TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE // 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
tables = null; if (!params.isSetType() || params.getType() == null || params.getType().isEmpty()) { @@ -381,9 +381,9 @@ public TDescribeTableResult describeTable(TDescribeTableParams params) throws TE 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 TMasterOpResult forward(TMasterOpRequest params) throws TException { 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 @@ private TLoadTxnBeginResult loadTxnBeginImpl(TLoadTxnBeginRequest request, Strin // 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 @@ private TLoadTxnBeginResult loadTxnBeginImpl(TLoadTxnBeginRequest request, Strin 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 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce // 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 @@ private boolean loadTxnCommitImpl(TLoadTxnCommitRequest request) throws UserExce } 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 @@ private void loadTxnRollbackImpl(TLoadTxnRollbackRequest request) throws UserExc 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 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) 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 @@ private TExecPlanFragmentParams streamLoadPutImpl(TStreamLoadPutRequest request) 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 020712288c9b7a..e34366fdbf142f 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 void alterColumnStatistics(AlterColumnStatsStmt stmt) throws AnalysisExce public List> 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> 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 @@ private Table validateTableName(TableName dbTableName) throws AnalysisException 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 bc5a9ffc2994e0..5e5fc150cae350 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 long getBackendReportVersion(long backendId) { } 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 c73217c43d956e..ad907783cb84d1 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 @@ protected void exec() { } 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 7f5fc54831d9f1..cffd7722e50d13 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 @@ protected void createEtlRequest() throws Exception { 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 @@ private Map createEtlPartitions() throws LoadException 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 7262e2eaa5452a..43b9a9ced4e26c 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 @@ protected void exec() { // 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 @@ private void tryUpdateLoading() { try { for (Entry 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 idToPartitionLoadInfo = tableLoadInfo.getIdToPartitionLoadInfo(); @@ -233,10 +230,7 @@ protected Map getTabletLoadInfos(Map idToTableLoadInfo = job.getIdToTableLoadInfo(); for (Entry 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 5242ca0137f5ea..0d2f699244f9cd 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 @@ protected void exec() { // 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 23b42eea47f463..ee04b403b14986 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 long beginTransaction(List tableIdList, String label, TUniqueId req 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 void commitTransaction(List
tableList, long transactionId, List tableList, long transactionId, List errorReplicaIds = Sets.newHashSet(); Set 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 void finishTransaction(long transactionId, Set errorReplicaIds) thr 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 void finishTransaction(long transactionId, Set errorReplicaIds) thr 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 @@ protected void unprotectedCommitTransaction(TransactionState transactionState, S 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 List> getSingleTranInfo(long dbId, long txnId) throws Analys List> infos = new ArrayList>(); 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 List> getSingleTranInfo(long dbId, long txnId) throws Analys // check auth Set 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 @@ private void updateCatalogAfterCommitted(TransactionState transactionState, Data Set 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 @@ private boolean updateCatalogAfterVisible(TransactionState transactionState, Dat Set 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 void removeExpiredAndTimeoutTxns(long currentMillis) { } } - 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 f62770b772c8d0..22b9a270baa461 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 @@ private void updateAllDatabaseUsedDataQuota() { List 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 83ebf070a32d19..e3416d24c8b183 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 void setEditLog(EditLog editLog) { // 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 List> getDbInfo() { for (long dbId : dbIds) { List info = new ArrayList(); info.add(dbId); - Database db = Catalog.getCurrentCatalog().getDb(dbId); + Database db = Catalog.getCurrentCatalog().getDbNullable(dbId); if (db == null) { continue; } @@ -514,10 +514,7 @@ public TWaitingTxnStatusResult getWaitingTxnStatus(TWaitingTxnStatusRequest requ 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 291ab881a19218..cd7d6632a3f50a 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 @@ private void publishVersion() throws UserException { 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 @@ private void publishVersion() throws UserException { 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 1b28317a368871..3c524a1a7aabff 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.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 void testSchemaChange() throws Exception { System.out.println(showResultSet.getResultRows()); } - private void waitAlterJobDone(Map alterJobs) throws InterruptedException { + private void waitAlterJobDone(Map 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 @@ private void waitAlterJobDone(Map alterJobs) throws Interrupte 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 void testRollup() throws Exception { @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 a206165dccc371..2a77d2f34c0ab3 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 void alterTableWithEnableFeature() throws Exception { @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 void testConflictAlterOperations() throws Exception { "'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 void testConflictAlterOperations() throws Exception { 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 void testConflictAlterOperations() throws Exception { // 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 void testDynamicPartitionDropAndAdd() throws Exception { 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 alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2(); if (rollupJob) { alterJobs = Catalog.getCurrentCatalog().getRollupHandler().getAlterJobsV2(); @@ -438,8 +438,8 @@ private void waitSchemaChangeJobDone(boolean rollupJob) throws InterruptedExcept } 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 void testReplaceTable() throws Exception { 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> 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 void testReplaceTable() throws Exception { // 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 void testReplaceTable() throws Exception { 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 void testReplaceTable() throws Exception { // 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 void testReplaceTable() throws Exception { 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 void testModifyBucketNum() throws Exception { "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 void testExternalTableAlterOperations() throws Exception { // 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 void testExternalTableAlterOperations() throws Exception { // 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 void testExternalTableAlterOperations() throws Exception { // 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 898deb33a21d29..2ccd1bc65f6f7e 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 void testBatchRollup() throws Exception { Map 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 void testCancelBatchRollup() throws Exception { Assert.assertEquals(3, alterJobs.size()); List 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 7293740e4fa85a..8a6f8f5d56f510 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 void testRunRollupJobConcurrentLimit() throws UserException { ArrayList 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 alterJobsV2 = materializedViewHandler.getAlterJobsV2(); @@ -176,8 +176,8 @@ public void testAddSchemaChange() throws UserException { MaterializedViewHandler materializedViewHandler = Catalog.getCurrentCatalog().getRollupHandler(); ArrayList 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 alterJobsV2 = materializedViewHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); @@ -195,8 +195,8 @@ public void testSchemaChange1() throws Exception { // add a rollup job ArrayList 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 alterJobsV2 = materializedViewHandler.getAlterJobsV2(); @@ -299,8 +299,8 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { // add a rollup job ArrayList 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 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 d299de2b6ca9f9..864416dff17c40 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 void testAddSchemaChange() throws UserException { SchemaChangeHandler schemaChangeHandler = Catalog.getCurrentCatalog().getSchemaChangeHandler(); ArrayList 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 alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); Assert.assertEquals(1, alterJobsV2.size()); @@ -154,8 +154,8 @@ public void testSchemaChange1() throws Exception { // add a schema change job ArrayList 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 alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); @@ -230,8 +230,8 @@ public void testSchemaChangeWhileTabletNotStable() throws Exception { // add a schema change job ArrayList 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 alterJobsV2 = schemaChangeHandler.getAlterJobsV2(); @@ -317,7 +317,7 @@ public void testModifyDynamicPartitionNormal() throws UserException { 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 void modifyDynamicPartitionWithoutTableProperty(String propertyKey, Strin 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 d222a6abf0c306..61587052dfc19b 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 static Catalog fetchAdminCatalog() { 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 static Database mockDb(String name) { 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 static Catalog fetchBlockCatalog() { 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 static Catalog fetchBlockCatalog() { minTimes = 0; result = Lists.newArrayList("testCluster:testDb"); - catalog.getDb("emptyCluster"); + catalog.getDbNullable("emptyCluster"); minTimes = 0; result = null; } @@ -388,7 +388,7 @@ public static Analyzer fetchEmptyDbAnalyzer() { 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 static Analyzer fetchTableAnalyzer() { 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 static Analyzer fetchTableAnalyzer() { 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 816600d13c5441..c67cf8cd4d09f8 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 void testShowReplicaDistribution() throws Exception { 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 df0f13ffd2de3f..64661209e35178 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.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 @@ PaloAuth getAuth() { 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 @@ String getClusterName() { } @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 fa029ac739e0d3..de3ef16cf6381a 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 void setUp() { properties = Maps.newHashMap(); new Expectations() { { - catalog.getDb("testCluster:testDb"); + catalog.getDbNullable("testCluster:testDb"); minTimes = 0; result = database; @@ -84,7 +84,7 @@ public void setUp() { 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 160773307785bf..bce2e072353e77 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 Catalog getCurrentCatalog() { }; 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 85395ebcf2debd..dee1b0f319b092 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 void setUp() throws AnalysisException { 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 aa9f50578b7656..e105863e064fa4 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 void setUp() { f.setAccessible(true); Multimap 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 a9209e098ce23a..325eaf9002e545 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 void setUp() throws AnalysisException { 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 ce464ce2dde6f0..bee5637c1ef9ab 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 void testGlobalVariable() { @Test public void testTableNameLowerCase() { - Set tableNames = Catalog.getCurrentCatalog().getDb("default_cluster:db1").getTableNamesWithLock(); + Set 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 400ea65e0ec3dc..3bf64fa947fb4d 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 void testGlobalVariable() { @Test public void testTableNameLowerCase() { - Set tableNames = Catalog.getCurrentCatalog().getDb("default_cluster:db1").getTableNamesWithLock(); + Set 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 e37a1292896a6e..c5053165560126 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 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 void setUp() throws IOException { f.setAccessible(true); Multimap 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 ffc51965ce02d7..84edd92d159733 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.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 void setUp() { } }; - 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 void testInit() { } @Test - public void testCreateAndDropRepository() { + public void testCreateAndDropRepository() throws Exception { new Expectations() { { editLog.logCreateRepository((Repository) any); @@ -206,8 +199,8 @@ public Status listSnapshots(List snapshotNames) { } @Mock - public Status getSnapshotInfoFile(String label, String backupTimestamp, List infos) { - OlapTable tbl = (OlapTable) db.getTable(CatalogMocker.TEST_TBL_NAME); + public Status getSnapshotInfoFile(String label, String backupTimestamp, List infos) throws Exception { + OlapTable tbl = (OlapTable) db.getTableOrMetaException(CatalogMocker.TEST_TBL_NAME); List
tbls = Lists.newArrayList(); tbls.add(tbl); List resources = Lists.newArrayList(); @@ -218,7 +211,7 @@ public Status getSnapshotInfoFile(String label, String backupTimestamp, List 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 Status getSnapshotInfoFile(String label, String backupTimestamp, List tblRefs = Lists.newArrayList(); @@ -259,17 +247,12 @@ public Status getSnapshotInfoFile(String label, String backupTimestamp, List snapshotFiles = Lists.newArrayList(); request.setSnapshotFiles(snapshotFiles); @@ -297,20 +280,12 @@ public Status getSnapshotInfoFile(String label, String backupTimestamp, List tblRefs2 = Lists.newArrayList(); @@ -320,19 +295,9 @@ public Status getSnapshotInfoFile(String label, String backupTimestamp, List 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 void testRunNormal() { Assert.assertNotNull(olapTable); Assert.assertNotNull(restoreMetaInfo.getTable(UnitTestUtil.TABLE_NAME)); List 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 da9166c17a5f0b..230bdab60074cf 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.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 static SystemInfoService fetchSystemInfoService() { 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 static Catalog fetchAdminCatalog() { 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 bf7fdf6c2e06c8..dd700583871c5a 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 Repository getRepo(long repoId) { 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 void setUp() throws AnalysisException { new Expectations() { { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = db; @@ -218,7 +218,7 @@ boolean await(long timeout, TimeUnit unit) { 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 void testRun() { } @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 void testSignature() { 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 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 d4dff175a6c26c..8f8b60360701b2 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 static void tearDown() { @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> 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 a85a5d21adad1e..1d545459e818dd 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 void testRenameTable() throws Exception { 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 void testRenameTable() throws Exception { 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 0ec5a1b352bef3..8e2eb77692542c 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.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 static Catalog createTestCatalog() throws InstantiationException, Illegal } public static boolean compareCatalog(Catalog masterCatalog, Catalog slaveCatalog) { - Database masterDb = masterCatalog.getDb(testDb1); - Database slaveDb = slaveCatalog.getDb(testDb1); - List
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 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
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 allTablets = masterIndex.getTablets(); - for (Tablet masterTablet : allTablets) { - Tablet slaveTablet = slaveIndex.getTablet(masterTablet.getId()); - if (slaveTablet == null) { + List allMaterializedIndices = masterPartition.getMaterializedIndices(IndexExtState.ALL); + for (MaterializedIndex masterIndex : allMaterializedIndices) { + MaterializedIndex slaveIndex = slavePartition.getIndex(masterIndex.getId()); + if (slaveIndex == null) { return false; } - List 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 allTablets = masterIndex.getTablets(); + for (Tablet masterTablet : allTablets) { + Tablet slaveTablet = slaveIndex.getTablet(masterTablet.getId()); + if (slaveTablet == null) { return false; } + List 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 3accf552267889..a4afb907f62ddc 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 void testCreateOneTable() throws Exception { ");"); 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.>getField(index, "group2Tables").size()); Assert.assertEquals(1, index.getAllGroupIds().size()); @@ -159,9 +159,9 @@ public void testCreateTwoTableWithSameGroup() throws Exception { ");"); 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.>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 cbd829ee333b29..0bfe7fdc548a48 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 void test() throws Exception { 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 c744ef040bb0e4..0e4f90a06a538e 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 void test() throws Exception { 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 858eb763d40b37..127dd65c26ed09 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 @@ private static void checkCreateOlapTableLike(String createTableSql, String creat 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 @@ private static void checkCreateMysqlTableLike(String createTableSql, String crea 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 6af49d25102674..71f3b6f434c3ca 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 void testNormal() throws DdlException { + "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 04c24bf909d627..141e8e1a75223d 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 void testNormal() throws DdlException { () -> 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 ac491708e6775e..161d2024c8dee5 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.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.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 void getTableOrThrowExceptionTest() throws MetaNotFoundException { 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 void createAndDropPartitionTest() { // 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 8a75300fac59d0..d695fa5bb4f5d4 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 @@ private static void createTable(String sql) throws Exception { @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 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 void testNormalDropDb() throws Exception { @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 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 ba2ef863f0db03..43f18cefd5cd68 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 @@ private static void dropPartition(String sql) throws Exception { @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 void testNormalDropPartition() throws Exception { @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 void testForceDropPartition() throws Exception { @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 4c12e5380ed735..fc543fea1ca75d 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 @@ private static void dropTable(String sql) throws Exception { @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 void testNormalDropTable() throws Exception { 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 754a44eac4f9b7..95852aea47ef34 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 void testNormal() throws Exception { "\"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 void testSetDynamicPartitionReplicationNum() throws Exception { "\"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 void testCreateDynamicPartitionImmediately() throws Exception { "\"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 partitionIterator = emptyDynamicTable.getAllPartitions().iterator(); @@ -539,7 +539,7 @@ public void testFillHistoryDynamicPartition() throws Exception { "\"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 partitionIterator = emptyDynamicTable.getAllPartitions().iterator(); @@ -676,8 +676,8 @@ public void testFillHistoryDynamicPartition3() throws Exception { ");"; // 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 void testFillHistoryDynamicPartitionWithHistoryPartitionNum() throws Exce "\"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 tableProperties = emptyDynamicTable.getTableProperty().getProperties(); Assert.assertEquals(14, emptyDynamicTable.getAllPartitions().size()); // never delete the old partitions @@ -747,7 +747,7 @@ public void testAllTypeDynamicPartition() throws Exception { "\"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 partitionIterator = emptyDynamicTable.getAllPartitions().iterator(); @@ -774,7 +774,7 @@ public void testAllTypeDynamicPartition() throws Exception { "\"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 void testAllTypeDynamicPartition() throws Exception { "\"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 void testAllTypeDynamicPartition() throws Exception { "\"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 void testAllTypeDynamicPartition() throws Exception { "\"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 void testAllTypeDynamicPartition() throws Exception { "\"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 void testHourDynamicPartitionWithIntType() throws Exception { @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 void testHotPartitionNum() throws Exception { "\"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 idToDataProperty = new TreeMap<>(partitionInfo.idToDataProperty); Assert.assertEquals(7, idToDataProperty.size()); @@ -970,7 +970,7 @@ public void testHotPartitionNum() throws Exception { "\"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 void testHotPartitionNum() throws Exception { "\"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 void testHotPartitionNum() throws Exception { "\"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 void testHotPartitionNum() throws Exception { "\"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 void testHotPartitionNum() throws Exception { "\"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 void testHotPartitionNum() throws Exception { "\"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 1cf7fdd338ae20..0a808c9bd1711f 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 void testNormal() throws IOException { 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 bb348f12ff3fbb..d8694b5c581703 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.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 static void setUp() throws NoSuchMethodException, SecurityException, Inst } @Before - public void before() { + public void before() throws Exception { new Expectations() { { @@ -74,7 +72,7 @@ public void before() { 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 e2ca0e8b28dc98..a69762d0a873c2 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 @@ private static void recoverPartition(String db, String tbl, String part) throws } 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 268ef36d1b68fd..56a0bdef7f10a7 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 void testForMultiPartitionTable() throws Exception { 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 void testForStrictRangeCheck() throws Exception { 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 void testForListPartitionTable() throws Exception { 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 void testForMultiListPartitionTable() throws Exception { 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 a6ea1e490ca5c8..537fefc88df691 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.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 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 void setUp() throws AnalysisException { minTimes = 0; result = db.getId(); - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); + minTimes = 0; + result = db; + + catalog.getDbOrException(anyLong, (Function) 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 aa4a9709e510f3..6a24b2a14a7317 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 void setUp() throws IOException { 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 aa9930a8adfd33..fcc5fa6f7c47f1 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 void testRegister() { public void testLookupNormal() throws AnalysisException { new Expectations(catalog) { { - catalog.getDb("db1"); + catalog.getDbNullable("db1"); minTimes = 0; result = db1; - catalog.getDb("db2"); + catalog.getDbNullable("db2"); minTimes = 0; result = db2; - catalog.getDb("db3"); + catalog.getDbNullable("db3"); minTimes = 0; result = null; - catalog.getDb(db1.getId()); + catalog.getDbNullable(db1.getId()); minTimes = 0; result = db1; - catalog.getDb(db2.getId()); + catalog.getDbNullable(db2.getId()); minTimes = 0; result = db2; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = null; } @@ -150,27 +150,27 @@ public void testFetchResultNormal() throws AnalysisException { minTimes = 0; result = Lists.newArrayList("db1", "db2"); - catalog.getDb("db1"); + catalog.getDbNullable("db1"); minTimes = 0; result = db1; - catalog.getDb("db2"); + catalog.getDbNullable("db2"); minTimes = 0; result = db2; - catalog.getDb("db3"); + catalog.getDbNullable("db3"); minTimes = 0; result = null; - catalog.getDb(db1.getId()); + catalog.getDbNullable(db1.getId()); minTimes = 0; result = db1; - catalog.getDb(db2.getId()); + catalog.getDbNullable(db2.getId()); minTimes = 0; result = db2; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = null; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java index 2333dd4f097e54..640f290aaa0e13 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java @@ -17,7 +17,9 @@ package org.apache.doris.common.util; -import mockit.MockUp; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; import org.apache.doris.analysis.CreateFileStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; @@ -26,15 +28,10 @@ import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.common.util.SmallFileMgr.SmallFile; import org.apache.doris.persist.EditLog; - import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; -import mockit.Expectations; -import mockit.Injectable; -import mockit.Mocked; - public class SmallFileMgrTest { @Mocked @@ -52,7 +49,7 @@ public void test(@Injectable CreateFileStmt stmt1, @Injectable CreateFileStmt st db.getId(); minTimes = 0; result = 1L; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = db; stmt1.getDbName(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/external/elasticsearch/EsShardPartitionsTest.java b/fe/fe-core/src/test/java/org/apache/doris/external/elasticsearch/EsShardPartitionsTest.java index 46ea20d8def762..23975a5688b70e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/external/elasticsearch/EsShardPartitionsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/external/elasticsearch/EsShardPartitionsTest.java @@ -31,8 +31,8 @@ public class EsShardPartitionsTest extends EsTestCase { @Test public void testPartition() throws Exception { EsTable esTable = (EsTable) Catalog.getCurrentCatalog() - .getDb(CatalogTestUtil.testDb1) - .getTable(CatalogTestUtil.testEsTableId1); + .getDbOrMetaException(CatalogTestUtil.testDb1) + .getTableOrMetaException(CatalogTestUtil.testEsTableId1); EsShardPartitions esShardPartitions = EsShardPartitions.findShardPartitions("doe", loadJsonFromFile("data/es/test_search_shards.json")); EsTablePartitions esTablePartitions = EsTablePartitions.fromShardPartitions(esTable, esShardPartitions); diff --git a/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java b/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java index 14abfbce8f3d8e..bf82315aae0802 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/http/DorisHttpTestCase.java @@ -204,11 +204,11 @@ private static Catalog newDelegateCatalog() { minTimes = 0; result = paloAuth; - catalog.getDb(db.getId()); + catalog.getDbNullable(db.getId()); minTimes = 0; result = db; - catalog.getDb("default_cluster:" + DB_NAME); + catalog.getDbNullable("default_cluster:" + DB_NAME); minTimes = 0; result = db; @@ -216,11 +216,11 @@ private static Catalog newDelegateCatalog() { minTimes = 0; result = true; - catalog.getDb("default_cluster:emptyDb"); + catalog.getDbNullable("default_cluster:emptyDb"); minTimes = 0; result = null; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = new Database(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/http/TableQueryPlanActionTest.java b/fe/fe-core/src/test/java/org/apache/doris/http/TableQueryPlanActionTest.java index 3e19e7def22338..9f9c436a3b9af6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/http/TableQueryPlanActionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/http/TableQueryPlanActionTest.java @@ -133,7 +133,6 @@ public void testNotOlapTableFailure() throws IOException { JSONObject jsonObject = new JSONObject(respStr); Assert.assertEquals(403, jsonObject.getInt("status")); String exception = jsonObject.getString("exception"); - Assert.assertNotNull(exception); - Assert.assertTrue(exception.startsWith("only support OlapTable currently")); + Assert.assertTrue(exception.contains("table type is not OLAP")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java index 8ee75c3092058a..035c3c5ba2a413 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/DeleteHandlerTest.java @@ -105,7 +105,7 @@ public class DeleteHandlerTest { private ConnectContext connectContext = new ConnectContext(); @Before - public void setUp() { + public void setUp() throws Exception { FeConstants.runningUnitTest = true; globalTransactionMgr = new GlobalTransactionMgr(catalog); @@ -113,12 +113,7 @@ public void setUp() { deleteHandler = new DeleteHandler(); auth = AccessTestUtil.fetchAdminAccess(); analyzer = AccessTestUtil.fetchAdminAnalyzer(false); - try { - db = CatalogMocker.mockDb(); - } catch (AnalysisException e) { - e.printStackTrace(); - Assert.fail(); - } + db = CatalogMocker.mockDb(); TabletMeta tabletMeta = new TabletMeta(DB_ID, TBL_ID, PARTITION_ID, TBL_ID, 0, null); invertedIndex.addTablet(TABLET_ID, tabletMeta); invertedIndex.addReplica(TABLET_ID, new Replica(REPLICA_ID_1, BACKEND_ID_1, 0, Replica.ReplicaState.NORMAL)); @@ -136,11 +131,11 @@ public void logInsertTransactionState(TransactionState transactionState) { new Expectations() { { - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = db; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = db; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java index 440ac0b32b7051..c4c7cbee6bcb8c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/LoadCheckerTest.java @@ -84,11 +84,11 @@ public void setUp() { db = UnitTestUtil.createDb(dbId, tableId, partitionId, indexId, tabletId, backendId, 1L, 0L); new Expectations() { { - catalog.getDb(dbId); + catalog.getDbNullable(dbId); minTimes = 0; result = db; - catalog.getDb(db.getFullName()); + catalog.getDbNullable(db.getFullName()); minTimes = 0; result = db; @@ -259,7 +259,7 @@ public void testRunLoadingJobs() throws Exception { job.setDbId(dbId); etlJobs.add(job); // set table family load infos - OlapTable table = (OlapTable) db.getTable(tableId); + OlapTable table = (OlapTable) db.getTableOrMetaException(tableId); Partition partition = table.getPartition(partitionId); long newVersion = partition.getVisibleVersion() + 1; long newVersionHash = 1L; @@ -340,7 +340,7 @@ public void testRunQuorumFinishedJobs() throws Exception { job.setDbId(dbId); etlJobs.add(job); // set table family load infos - OlapTable table = (OlapTable) db.getTable(tableId); + OlapTable table = (OlapTable) db.getTableOrMetaException(tableId); Partition partition = table.getPartition(partitionId); long newVersion = partition.getVisibleVersion() + 1; long newVersionHash = 0L; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java index 5ef4dcba4afda8..753df1af4253aa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/BrokerLoadJobTest.java @@ -67,6 +67,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -96,7 +97,7 @@ public void testFromLoadStmt(@Injectable LoadStmt loadStmt, labelName.getDbName(); minTimes = 0; result = databaseName; - catalog.getDb(databaseName); + catalog.getDbNullable(databaseName); minTimes = 0; result = database; loadStmt.getDataDescriptions(); @@ -105,7 +106,7 @@ public void testFromLoadStmt(@Injectable LoadStmt loadStmt, dataDescription.getTableName(); minTimes = 0; result = tableName; - database.getTable(tableName); + database.getTableNullable(tableName); minTimes = 0; result = null; } @@ -147,7 +148,7 @@ public void testFromLoadStmt2(@Injectable LoadStmt loadStmt, labelName.getLabelName(); minTimes = 0; result = label; - catalog.getDb(databaseName); + catalog.getDbNullable(databaseName); minTimes = 0; result = database; loadStmt.getDataDescriptions(); @@ -156,7 +157,7 @@ public void testFromLoadStmt2(@Injectable LoadStmt loadStmt, dataDescription.getTableName(); minTimes = 0; result = tableName; - database.getTable(tableName); + database.getTableNullable(tableName); minTimes = 0; result = olapTable; dataDescription.getPartitionNames(); @@ -218,10 +219,10 @@ public void testGetTableNames(@Injectable BrokerFileGroupAggInfo fileGroupAggInf result = Sets.newHashSet(1L); catalog.getDb(anyLong); minTimes = 0; - result = database; + result = Optional.of(database); database.getTable(1L); minTimes = 0; - result = table; + result = Optional.of(table); table.getName(); minTimes = 0; result = tableName; @@ -229,7 +230,7 @@ public void testGetTableNames(@Injectable BrokerFileGroupAggInfo fileGroupAggInf }; Assert.assertEquals(1, brokerLoadJob.getTableNamesForShow().size()); - Assert.assertEquals(true, brokerLoadJob.getTableNamesForShow().contains(tableName)); + Assert.assertTrue(brokerLoadJob.getTableNamesForShow().contains(tableName)); } @Test @@ -309,13 +310,13 @@ public void testPendingTaskOnFinished(@Injectable BrokerPendingTaskAttachment at attachment.getTaskId(); minTimes = 0; result = taskId; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; fileGroupAggInfo.getAggKeyToFileGroups(); minTimes = 0; result = aggKeyToFileGroups; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = olapTable; catalog.getNextId(); @@ -480,7 +481,7 @@ public void testLoadingTaskOnFinished(@Injectable BrokerLoadingTaskAttachment at attachment1.getTaskId(); minTimes = 0; result = 1L; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/InsertLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/InsertLoadJobTest.java index 8a5269238734fe..cba2f717a55172 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/InsertLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/InsertLoadJobTest.java @@ -26,6 +26,7 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Optional; import java.util.Set; import mockit.Expectations; @@ -43,16 +44,16 @@ public void testGetTableNames(@Mocked Catalog catalog, new Expectations() { { catalog.getDb(anyLong); - result = database; + result = Optional.of(database); database.getTable(anyLong); - result = table; + result = Optional.of(table); table.getName(); result = tableName; } }; Set tableNames = insertLoadJob.getTableNamesForShow(); Assert.assertEquals(1, tableNames.size()); - Assert.assertEquals(true, tableNames.contains(tableName)); + Assert.assertTrue(tableNames.contains(tableName)); Assert.assertEquals(JobState.FINISHED, insertLoadJob.getState()); Assert.assertEquals(Integer.valueOf(100), Deencapsulation.getField(insertLoadJob, "progress")); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadJobTest.java index 5afe467e661ad8..c4dc3f97c124eb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadJobTest.java @@ -57,26 +57,6 @@ public static void start() { MetricRepo.init(); } - @Test - public void testGetDbNotExists(@Mocked Catalog catalog) { - LoadJob loadJob = new BrokerLoadJob(); - Deencapsulation.setField(loadJob, "dbId", 1L); - new Expectations() { - { - catalog.getDb(1L); - minTimes = 0; - result = null; - } - }; - - try { - loadJob.getDb(); - Assert.fail(); - } catch (MetaNotFoundException e) { - } - } - - @Test public void testSetJobPropertiesWithErrorTimeout() { Map jobProperties = Maps.newHashMap(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java index d8ae0a52827a7e..d33f31027ced1a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/LoadManagerTest.java @@ -91,7 +91,7 @@ public void testCreateHadoopJob(@Injectable LoadStmt stmt, labelName.getLabelName(); minTimes = 0; result = "label1"; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = database; database.getId(); @@ -117,10 +117,10 @@ public void testSerializationNormal(@Mocked Catalog catalog, @Injectable Table table) throws Exception { new Expectations(){ { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = table; table.getName(); @@ -152,10 +152,10 @@ public void testSerializationWithJobRemoved(@Mocked MetaContext metaContext, @Injectable Table table) throws Exception { new Expectations(){ { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = table; table.getName(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java index 1046e5dcd6bc88..09c21dbaa0e819 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadJobTest.java @@ -129,7 +129,7 @@ public void setUp() { public void testCreateFromLoadStmt(@Mocked Catalog catalog, @Injectable LoadStmt loadStmt, @Injectable DataDescription dataDescription, @Injectable LabelName labelName, @Injectable Database db, @Injectable OlapTable olapTable, - @Injectable ResourceMgr resourceMgr) { + @Injectable ResourceMgr resourceMgr) throws Exception { List dataDescriptionList = Lists.newArrayList(); dataDescriptionList.add(dataDescription); Map resourceProperties = Maps.newHashMap(); @@ -143,12 +143,10 @@ public void testCreateFromLoadStmt(@Mocked Catalog catalog, @Injectable LoadStmt new Expectations() { { - catalog.getDb(dbName); + catalog.getDbOrDdlException(dbName); result = db; catalog.getResourceMgr(); result = resourceMgr; - db.getTable(tableName); - result = olapTable; db.getId(); result = dbId; loadStmt.getLabel(); @@ -174,29 +172,25 @@ public void testCreateFromLoadStmt(@Mocked Catalog catalog, @Injectable LoadStmt } }; - try { - Assert.assertTrue(resource.getSparkConfigs().isEmpty()); - resourceDesc.analyze(); - BulkLoadJob bulkLoadJob = BulkLoadJob.fromLoadStmt(loadStmt); - SparkLoadJob sparkLoadJob = (SparkLoadJob) bulkLoadJob; - // check member - Assert.assertEquals(dbId, bulkLoadJob.dbId); - Assert.assertEquals(label, bulkLoadJob.label); - Assert.assertEquals(JobState.PENDING, bulkLoadJob.getState()); - Assert.assertEquals(EtlJobType.SPARK, bulkLoadJob.getJobType()); - Assert.assertEquals(resourceName, sparkLoadJob.getResourceName()); - Assert.assertEquals(-1L, sparkLoadJob.getEtlStartTimestamp()); - - // check update spark resource properties - Assert.assertEquals(broker, bulkLoadJob.brokerDesc.getName()); - Assert.assertEquals("user0", bulkLoadJob.brokerDesc.getProperties().get("username")); - Assert.assertEquals("password0", bulkLoadJob.brokerDesc.getProperties().get("password")); - SparkResource sparkResource = Deencapsulation.getField(sparkLoadJob, "sparkResource"); - Assert.assertTrue(sparkResource.getSparkConfigs().containsKey("spark.executor.memory")); - Assert.assertEquals("1g", sparkResource.getSparkConfigs().get("spark.executor.memory")); - } catch (DdlException | AnalysisException e) { - Assert.fail(e.getMessage()); - } + Assert.assertTrue(resource.getSparkConfigs().isEmpty()); + resourceDesc.analyze(); + BulkLoadJob bulkLoadJob = BulkLoadJob.fromLoadStmt(loadStmt); + SparkLoadJob sparkLoadJob = (SparkLoadJob) bulkLoadJob; + // check member + Assert.assertEquals(dbId, bulkLoadJob.dbId); + Assert.assertEquals(label, bulkLoadJob.label); + Assert.assertEquals(JobState.PENDING, bulkLoadJob.getState()); + Assert.assertEquals(EtlJobType.SPARK, bulkLoadJob.getJobType()); + Assert.assertEquals(resourceName, sparkLoadJob.getResourceName()); + Assert.assertEquals(-1L, sparkLoadJob.getEtlStartTimestamp()); + + // check update spark resource properties + Assert.assertEquals(broker, bulkLoadJob.brokerDesc.getName()); + Assert.assertEquals("user0", bulkLoadJob.brokerDesc.getProperties().get("username")); + Assert.assertEquals("password0", bulkLoadJob.brokerDesc.getProperties().get("password")); + SparkResource sparkResource = Deencapsulation.getField(sparkLoadJob, "sparkResource"); + Assert.assertTrue(sparkResource.getSparkConfigs().containsKey("spark.executor.memory")); + Assert.assertEquals("1g", sparkResource.getSparkConfigs().get("spark.executor.memory")); } @Test @@ -355,8 +349,6 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( result = status; handler.getEtlFilePaths(etlOutputPath, (BrokerDesc) any); result = filePathToSize; - catalog.getDb(dbId); - result = db; db.getTablesOnIdOrderOrThrowException((List) any); result = Lists.newArrayList(table); table.getId(); @@ -426,13 +418,6 @@ public void testUpdateEtlStatusFinishedAndCommitTransaction( @Test public void testSubmitTasksWhenStateFinished(@Mocked Catalog catalog, @Injectable String originStmt, @Injectable Database db) throws Exception { - new Expectations() { - { - catalog.getDb(dbId); - result = db; - } - }; - SparkLoadJob job = getEtlStateJob(originStmt); job.state = JobState.FINISHED; Set totalTablets = Deencapsulation.invoke(job, "submitPushTasks"); @@ -522,14 +507,8 @@ public void testSparkLoadJobPersist(@Mocked Catalog catalog, @Mocked Database db Maps.newHashMap()); new Expectations() { { - catalog.getDb(dbId); - result = db; catalog.getResourceMgr(); result = resourceMgr; - //db.getTable(anyLong); - //result = table; - //table.getName(); - //result = "table1"; resourceMgr.getResource(anyString); result = sparkResource; Catalog.getCurrentCatalogJournalVersion(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java index 491773b7764d8a..e2f88b8a7bf0f2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java @@ -105,12 +105,8 @@ public void testExecuteTask(@Injectable SparkLoadJob sparkLoadJob, new Expectations() { { - catalog.getDb(dbId); - result = database; sparkLoadJob.getHandle(); result = handle; - database.getTable(tableId); - result = table; table.getPartitions(); result = partitions; table.getIndexIdToSchema(); @@ -148,55 +144,6 @@ public void submitEtlJob(long loadJobId, String loadLabel, EtlJobConfig etlJobCo Assert.assertEquals(appId, attachment.getAppId()); } - @Test(expected = LoadException.class) - public void testNoDb(@Injectable SparkLoadJob sparkLoadJob, - @Injectable SparkResource resource, - @Injectable BrokerDesc brokerDesc, - @Mocked Catalog catalog) throws LoadException { - long dbId = 0L; - - new Expectations() { - { - catalog.getDb(dbId); - result = null; - } - }; - - SparkLoadPendingTask task = new SparkLoadPendingTask(sparkLoadJob, null, resource, brokerDesc); - task.init(); - } - - @Test(expected = LoadException.class) - public void testNoTable(@Injectable SparkLoadJob sparkLoadJob, - @Injectable SparkResource resource, - @Injectable BrokerDesc brokerDesc, - @Mocked Catalog catalog, - @Injectable Database database) throws LoadException { - long dbId = 0L; - long tableId = 1L; - - Map> aggKeyToFileGroups = Maps.newHashMap(); - List brokerFileGroups = Lists.newArrayList(); - DataDescription desc = new DataDescription("testTable", null, Lists.newArrayList("abc.txt"), - null, null, null, false, null); - BrokerFileGroup brokerFileGroup = new BrokerFileGroup(desc); - brokerFileGroups.add(brokerFileGroup); - BrokerFileGroupAggInfo.FileGroupAggKey aggKey = new BrokerFileGroupAggInfo.FileGroupAggKey(tableId, null); - aggKeyToFileGroups.put(aggKey, brokerFileGroups); - - new Expectations() { - { - catalog.getDb(dbId); - result = database; - database.getTable(tableId); - result = null; - } - }; - - SparkLoadPendingTask task = new SparkLoadPendingTask(sparkLoadJob, aggKeyToFileGroups, resource, brokerDesc); - task.init(); - } - @Test public void testRangePartitionHashDistribution(@Injectable SparkLoadJob sparkLoadJob, @Injectable SparkResource resource, @@ -253,10 +200,6 @@ public void testRangePartitionHashDistribution(@Injectable SparkLoadJob sparkLoa new Expectations() { { - catalog.getDb(dbId); - result = database; - database.getTable(tableId); - result = table; table.getPartitions(); result = partitions; table.getIndexIdToSchema(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index 7ce75a7c939338..568c0c859b5c02 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -241,30 +241,6 @@ public void testProcessTimeOutTasks(@Injectable GlobalTransactionMgr globalTrans }; } - @Test - public void testFromCreateStmtWithErrorTable(@Mocked Catalog catalog, - @Injectable Database database) throws LoadException { - CreateRoutineLoadStmt createRoutineLoadStmt = initCreateRoutineLoadStmt(); - RoutineLoadDesc routineLoadDesc = new RoutineLoadDesc(columnSeparator, null, null, null, null, - partitionNames, null, LoadTask.MergeType.APPEND, null); - Deencapsulation.setField(createRoutineLoadStmt, "routineLoadDesc", routineLoadDesc); - - new Expectations() { - { - database.getTable(tableNameString); - minTimes = 0; - result = null; - } - }; - - try { - KafkaRoutineLoadJob kafkaRoutineLoadJob = KafkaRoutineLoadJob.fromCreateStmt(createRoutineLoadStmt); - Assert.fail(); - } catch (UserException e) { - LOG.info(e.getMessage()); - } - } - @Test public void testFromCreateStmt(@Mocked Catalog catalog, @Injectable Database database, @@ -291,7 +267,7 @@ public void testFromCreateStmt(@Mocked Catalog catalog, new Expectations() { { - database.getTable(tableNameString); + database.getTableNullable(tableNameString); minTimes = 0; result = table; database.getId(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java index 7f0676fe987993..88465153340398 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java @@ -196,7 +196,7 @@ public void testGetShowInfo(@Mocked KafkaProgress kafkaProgress) { public void testUpdateWhileDbDeleted(@Mocked Catalog catalog) throws UserException { new Expectations() { { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = null; } @@ -213,10 +213,10 @@ public void testUpdateWhileTableDeleted(@Mocked Catalog catalog, @Injectable Database database) throws UserException { new Expectations() { { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = null; } @@ -238,10 +238,10 @@ public void testUpdateWhilePartitionChanged(@Mocked Catalog catalog, new Expectations() { { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = table; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java index f66b1972121688..0512e3eb0f0ad9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java @@ -542,7 +542,7 @@ public void testGetJobIncludeHistory(@Injectable RoutineLoadJob routineLoadJob1, routineLoadJob3.isFinal(); minTimes = 0; result = true; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = database; database.getId(); @@ -597,7 +597,7 @@ public void testPauseRoutineLoadJob(@Injectable PauseRoutineLoadStmt pauseRoutin pauseRoutineLoadStmt.getName(); minTimes = 0; result = ""; - catalog.getDb(""); + catalog.getDbNullable(""); minTimes = 0; result = database; database.getId(); @@ -657,7 +657,7 @@ public void testResumeRoutineLoadJob(@Injectable ResumeRoutineLoadStmt resumeRou resumeRoutineLoadStmt.getName(); minTimes = 0; result = ""; - catalog.getDb(""); + catalog.getDbNullable(""); minTimes = 0; result = database; database.getId(); @@ -701,7 +701,7 @@ public void testStopRoutineLoadJob(@Injectable StopRoutineLoadStmt stopRoutineLo stopRoutineLoadStmt.getName(); minTimes = 0; result = ""; - catalog.getDb(""); + catalog.getDbNullable(""); minTimes = 0; result = database; database.getId(); @@ -894,7 +894,7 @@ public void testAlterRoutineLoadJob(@Injectable StopRoutineLoadStmt stopRoutineL stopRoutineLoadStmt.getName(); minTimes = 0; result = ""; - catalog.getDb(""); + catalog.getDbNullable(""); minTimes = 0; result = database; database.getId(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java index 7228a2eafc99e2..f07280429503b2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java @@ -92,10 +92,10 @@ public void testNormalRunOneCycle(@Mocked Catalog catalog, routineLoadManager.getRoutineLoadJobByState(Sets.newHashSet(RoutineLoadJob.JobState.NEED_SCHEDULE)); minTimes = 0; result = routineLoadJobList; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(1L); + database.getTableNullable(1L); minTimes = 0; result = olapTable; systemInfoService.getClusterBackendIds(clusterName, true); @@ -151,7 +151,7 @@ public void functionTest(@Mocked Catalog catalog, catalog.getRoutineLoadManager(); minTimes = 0; result = routineLoadManager; - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; systemInfoService.getBackendIds(true); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/sync/SyncJobManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/sync/SyncJobManagerTest.java index 232b414ec91a98..7b08c757918e6f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/sync/SyncJobManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/sync/SyncJobManagerTest.java @@ -71,7 +71,7 @@ public void setUp() throws DdlException { catalog.getEditLog(); minTimes = 0; result = editLog; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = database; database.getId(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/sync/canal/CanalSyncJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/sync/canal/CanalSyncJobTest.java index 4d4a8d1485a84b..597636be7d44e0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/sync/canal/CanalSyncJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/sync/canal/CanalSyncJobTest.java @@ -91,11 +91,11 @@ public void setUp() { catalog = Deencapsulation.newInstance(Catalog.class); new Expectations(catalog) { { - catalog.getDb(10000L); + catalog.getDbNullable(10000L); minTimes = 0; result = database; - catalog.getDb("testDb"); + catalog.getDbNullable("testDb"); minTimes = 0; result = database; @@ -115,7 +115,7 @@ public void setUp() { minTimes = 0; result = dbId; - database.getTable("testTbl"); + database.getTableNullable("testTbl"); minTimes = 0; result = table; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/update/UpdateStmtExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/update/UpdateStmtExecutorTest.java index c70ddb5fe947cf..839ffd5a6e41d5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/update/UpdateStmtExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/update/UpdateStmtExecutorTest.java @@ -71,8 +71,6 @@ public void testFromUpdateStmt(@Injectable OlapTable olapTable, Deencapsulation.setField(updateStmt, "analyzer", analyzer); new Expectations() { { - catalog.getDb("db"); - result = db; db.getId(); result = 1; analyzer.getContext().queryId(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mysql/MysqlProtoTest.java b/fe/fe-core/src/test/java/org/apache/doris/mysql/MysqlProtoTest.java index e9ac2f03abc63c..89dcc62300c7c1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mysql/MysqlProtoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mysql/MysqlProtoTest.java @@ -82,7 +82,7 @@ boolean fakeCheckPassword(String remoteUser, String remoteHost, byte[] remotePas } }; - catalog.getDb(anyString); + catalog.getDbNullable(anyString); minTimes = 0; result = new Database(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/LoadJobV2PersistTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/LoadJobV2PersistTest.java index 9bd0434aa58832..a63c323497ac42 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/LoadJobV2PersistTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/LoadJobV2PersistTest.java @@ -64,10 +64,10 @@ public void testBrokerLoadJob(@Mocked Catalog catalog, new Expectations() { { - catalog.getDb(anyLong); + catalog.getDbNullable(anyLong); minTimes = 0; result = database; - database.getTable(anyLong); + database.getTableNullable(anyLong); minTimes = 0; result = table; table.getName(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java index 499ae626437b50..c26d8289995795 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/QueryPlanTest.java @@ -1073,8 +1073,8 @@ public void testBucketShuffleJoin() throws Exception { Deencapsulation.setField(connectContext.getSessionVariable(), "enableBucketShuffleJoin", true); // set data size and row count for the olap table - Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test"); - OlapTable tbl = (OlapTable) db.getTable("bucket_shuffle1"); + Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("bucket_shuffle1"); for (Partition partition : tbl.getPartitions()) { partition.updateVisibleVersionAndVersionHash(2, 0); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { @@ -1087,8 +1087,8 @@ public void testBucketShuffleJoin() throws Exception { } } - db = Catalog.getCurrentCatalog().getDb("default_cluster:test"); - tbl = (OlapTable) db.getTable("bucket_shuffle2"); + db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test"); + tbl = (OlapTable) db.getTableOrMetaException("bucket_shuffle2"); for (Partition partition : tbl.getPartitions()) { partition.updateVisibleVersionAndVersionHash(2, 0); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { @@ -1157,8 +1157,8 @@ public void testJoinWithMysqlTable() throws Exception { connectContext.setDatabase("default_cluster:test"); // set data size and row count for the olap table - Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test"); - OlapTable tbl = (OlapTable) db.getTable("jointest"); + Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("jointest"); for (Partition partition : tbl.getPartitions()) { partition.updateVisibleVersionAndVersionHash(2, 0); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { @@ -1204,8 +1204,8 @@ public void testJoinWithOdbcTable() throws Exception { connectContext.setDatabase("default_cluster:test"); // set data size and row count for the olap table - Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test"); - OlapTable tbl = (OlapTable) db.getTable("jointest"); + Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException("jointest"); for (Partition partition : tbl.getPartitions()) { partition.updateVisibleVersionAndVersionHash(2, 0); for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java index 25136c55da7ad3..95b58482dc05df 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/RepeatNodeTest.java @@ -64,12 +64,10 @@ public void setUp() throws Exception { f.setAccessible(true); Multimap 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/planner/StreamLoadScanNodeTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java index 28aafaf631a382..3a212756f6b8a5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/StreamLoadScanNodeTest.java @@ -692,7 +692,7 @@ public void testSequenceColumnWithSetColumns() throws UserException { new Expectations() { { - db.getTable(anyInt); + db.getTableNullable(anyInt); result = dstTable; minTimes = 0; dstTable.hasSequenceCol(); @@ -762,7 +762,7 @@ public void testSequenceColumnWithoutSetColumns() throws UserException { new Expectations() { { - db.getTable(anyInt); + db.getTableNullable(anyInt); result = dstTable; minTimes = 0; dstTable.hasSequenceCol(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java index 0bebcc4b3340bd..f4bc54e4b53035 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/PartitionCacheTest.java @@ -159,15 +159,15 @@ public SystemInfoService getCurrentSystemInfo() { minTimes = 0; result = auth; - catalog.getDb(fullDbName); + catalog.getDbNullable(fullDbName); minTimes = 0; result = db; - catalog.getDb(dbName); + catalog.getDbNullable(dbName); minTimes = 0; result = db; - catalog.getDb(db.getId()); + catalog.getDbNullable(db.getId()); minTimes = 0; result = db; diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java index bbb71108e5c2f2..19959056c3e2d8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/ShowExecutorTest.java @@ -159,7 +159,7 @@ public void setUp() throws Exception { db.readUnlock(); minTimes = 0; - db.getTable(anyString); + db.getTableNullable(anyString); minTimes = 0; result = table; } @@ -172,11 +172,11 @@ public void setUp() throws Exception { catalog = Deencapsulation.newInstance(Catalog.class); new Expectations(catalog) { { - catalog.getDb("testCluster:testDb"); + catalog.getDbNullable("testCluster:testDb"); minTimes = 0; result = db; - catalog.getDb("testCluster:emptyDb"); + catalog.getDbNullable("testCluster:emptyDb"); minTimes = 0; result = null; @@ -483,7 +483,7 @@ public void testShowColumnFromUnknownTable() throws AnalysisException { ShowExecutor executor = new ShowExecutor(ctx, stmt); expectedEx.expect(AnalysisException.class); - expectedEx.expectMessage("Unknown table 'testCluster:emptyDb.testTable'"); + expectedEx.expectMessage("Unknown database 'testCluster:emptyDb'"); executor.execute(); // empty table @@ -492,7 +492,7 @@ public void testShowColumnFromUnknownTable() throws AnalysisException { executor = new ShowExecutor(ctx, stmt); expectedEx.expect(AnalysisException.class); - expectedEx.expectMessage("Unknown table 'testCluster:testDb.emptyTable'"); + expectedEx.expectMessage("Unknown database 'testCluster:emptyDb'"); executor.execute(); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java index 0f08c9d7e374a0..9f3926d73a820c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/task/LoadEtlTaskTest.java @@ -90,11 +90,11 @@ public void testRunEtlTask(@Mocked DppScheduler dppScheduler) throws Exception { db = UnitTestUtil.createDb(dbId, tableId, partitionId, indexId, tabletId, backendId, 1L, 0L); new Expectations(catalog) { { - catalog.getDb(dbId); + catalog.getDbNullable(dbId); minTimes = 0; result = db; - catalog.getDb(db.getFullName()); + catalog.getDbNullable(db.getFullName()); minTimes = 0; result = db; @@ -114,7 +114,7 @@ public void testRunEtlTask(@Mocked DppScheduler dppScheduler) throws Exception { String cluster = Config.dpp_default_cluster; job.setClusterInfo(cluster, Load.clusterToDppConfig.get(cluster)); // set partition load infos - OlapTable table = (OlapTable) db.getTable(tableId); + OlapTable table = (OlapTable) db.getTableOrMetaException(tableId); Partition partition = table.getPartition(partitionId); Source source = new Source(new ArrayList()); List sources = Lists.newArrayList(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/LoadPendingTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/LoadPendingTaskTest.java index d882dc67c6b1b8..dec07d8f3f689f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/task/LoadPendingTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/task/LoadPendingTaskTest.java @@ -93,11 +93,11 @@ public void testRunPendingTask() throws Exception { // mock catalog new Expectations(catalog) { { - catalog.getDb(dbId); + catalog.getDbNullable(dbId); minTimes = 0; result = db; - catalog.getDb(db.getFullName()); + catalog.getDbNullable(db.getFullName()); minTimes = 0; result = db; @@ -122,7 +122,7 @@ public void testRunPendingTask() throws Exception { String cluster = Config.dpp_default_cluster; job.setClusterInfo(cluster, Load.clusterToDppConfig.get(cluster)); // set partition load infos - OlapTable table = (OlapTable) db.getTable(tableId); + OlapTable table = (OlapTable) db.getTableOrMetaException(tableId); table.setBaseIndexId(0L); Partition partition = table.getPartition(partitionId); Source source = new Source(new ArrayList()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java index 1caf243edf27eb..42f2e1ba3f3c62 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/DatabaseTransactionMgrTest.java @@ -101,7 +101,7 @@ public Map addTransactionToTransactionMgr() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId1, transTablets); masterTransMgr.finishTransaction(CatalogTestUtil.testDbId1, transactionId1, null); LabelToTxnId.put(CatalogTestUtil.testTxnLabel1, transactionId1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index 6efa5b7d7e76ee..bb2725f3fe745b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -174,13 +174,13 @@ public void testCommitTransaction1() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); // check status is committed assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); // check replica version - Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + Partition testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); // check partition version assertEquals(CatalogTestUtil.testStartVersion, testPartition.getVisibleVersion()); @@ -213,7 +213,7 @@ public void testCommitTransactionWithOneFailed() throws UserException { List transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); // follower catalog replay the transaction @@ -243,7 +243,7 @@ public void testCommitTransactionWithOneFailed() throws UserException { assertEquals(TransactionStatus.PREPARE, transactionState.getTransactionStatus()); } // check replica version - Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + Partition testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); // check partition version assertEquals(CatalogTestUtil.testStartVersion, testPartition.getVisibleVersion()); @@ -269,7 +269,7 @@ public void testCommitTransactionWithOneFailed() throws UserException { // check status is commit assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); // check replica version - testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); // check partition version assertEquals(CatalogTestUtil.testStartVersion, testPartition.getVisibleVersion()); @@ -360,7 +360,7 @@ LoadJobSourceType.ROUTINE_LOAD_TASK, new TxnCoordinator(TxnSourceType.BE, "be1") routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); Deencapsulation.setField(masterTransMgr.getDatabaseTransactionMgr(CatalogTestUtil.testDbId1), "idToRunningTransactionState", idToTransactionState); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(1L, Lists.newArrayList(testTable1), 1L, transTablets, txnCommitAttachment); RoutineLoadStatistic jobStatistic = Deencapsulation.getField(routineLoadJob,"jobStatistic"); @@ -428,7 +428,7 @@ LoadJobSourceType.ROUTINE_LOAD_TASK, new TxnCoordinator(TxnSourceType.BE, "be1") routineLoadManager.addRoutineLoadJob(routineLoadJob, "db"); Deencapsulation.setField(masterTransMgr.getDatabaseTransactionMgr(CatalogTestUtil.testDbId1), "idToRunningTransactionState", idToTransactionState); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(1L, Lists.newArrayList(testTable1), 1L, transTablets, txnCommitAttachment); // current total rows and error rows will be reset after job pause, so here they should be 0. @@ -459,7 +459,7 @@ public void testFinishTransaction() throws UserException { transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); transTablets.add(tabletCommitInfo3); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); TransactionState transactionState = fakeEditLog.getTransaction(transactionId); assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); @@ -469,7 +469,7 @@ public void testFinishTransaction() throws UserException { transactionState = fakeEditLog.getTransaction(transactionId); assertEquals(TransactionStatus.VISIBLE, transactionState.getTransactionStatus()); // check replica version - Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + Partition testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); // check partition version assertEquals(CatalogTestUtil.testStartVersion + 1, testPartition.getVisibleVersion()); @@ -492,7 +492,7 @@ public void testFinishTransaction() throws UserException { @Test public void testFinishTransactionWithOneFailed() throws UserException { TransactionState transactionState = null; - Partition testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + Partition testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); Tablet tablet = testPartition.getIndex(CatalogTestUtil.testIndexId1).getTablet(CatalogTestUtil.testTabletId1); FakeCatalog.setCatalog(masterCatalog); @@ -508,7 +508,7 @@ public void testFinishTransactionWithOneFailed() throws UserException { List transTablets = Lists.newArrayList(); transTablets.add(tabletCommitInfo1); transTablets.add(tabletCommitInfo2); - Table testTable1 = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1); + Table testTable1 = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1); masterTransMgr.commitTransaction(CatalogTestUtil.testDbId1, Lists.newArrayList(testTable1), transactionId, transTablets); // follower catalog replay the transaction @@ -583,7 +583,7 @@ public void testFinishTransactionWithOneFailed() throws UserException { // check status is commit assertEquals(TransactionStatus.COMMITTED, transactionState.getTransactionStatus()); // check replica version - testPartition = masterCatalog.getDb(CatalogTestUtil.testDbId1).getTable(CatalogTestUtil.testTableId1) + testPartition = masterCatalog.getDbOrMetaException(CatalogTestUtil.testDbId1).getTableOrMetaException(CatalogTestUtil.testTableId1) .getPartition(CatalogTestUtil.testPartition1); // check partition version assertEquals(CatalogTestUtil.testStartVersion + 1, testPartition.getVisibleVersion()); 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 a84a2e53b70eb1..866d0afba4e58f 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 @@ -17,44 +17,29 @@ package org.apache.doris.utframe; -import com.google.common.collect.ImmutableMap; import org.apache.doris.analysis.CreateDbStmt; import org.apache.doris.analysis.CreateTableStmt; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.DiskInfo; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; -import org.apache.doris.common.Pair; import org.apache.doris.planner.OlapScanNode; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.Planner; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; -import org.apache.doris.system.Backend; -import org.apache.doris.system.SystemInfoService; -import org.apache.doris.thrift.TNetworkAddress; -import org.apache.doris.utframe.MockedBackendFactory.DefaultBeThriftServiceImpl; -import org.apache.doris.utframe.MockedBackendFactory.DefaultHeartbeatServiceImpl; -import org.apache.doris.utframe.MockedBackendFactory.DefaultPBackendServiceImpl; import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException; import org.apache.doris.utframe.MockedFrontend.FeStartException; import org.apache.doris.utframe.MockedFrontend.NotInitException; - -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; -import java.nio.file.Files; import java.util.List; -import java.util.Map; import java.util.UUID; /* @@ -117,9 +102,8 @@ public void testCreateDbAndTable() throws Exception { CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr, ctx); Catalog.getCurrentCatalog().createTable(createTableStmt); // 4. get and test the created db and table - Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1"); - Assert.assertNotNull(db); - OlapTable tbl = (OlapTable) db.getTable("tbl1"); + Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:db1"); + OlapTable tbl = 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 906d6c1f8222cb..059708e465e1f9 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 @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.planner.OlapScanNode; @@ -88,9 +89,8 @@ public void testCreateDbAndTable() throws Exception { CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr, ctx); Catalog.getCurrentCatalog().createTable(createTableStmt); // 4. get and test the created db and table - Database db = Catalog.getCurrentCatalog().getDb("default_cluster:db1"); - Assert.assertNotNull(db); - OlapTable tbl = (OlapTable) db.getTable("tbl1"); + Database db = Catalog.getCurrentCatalog().getDbOrMetaException("default_cluster:db1"); + OlapTable tbl = db.getTableOrMetaException("tbl1", Table.TableType.OLAP); tbl.readLock(); try { Assert.assertNotNull(tbl); @@ -116,7 +116,7 @@ public void testCreateDbAndTable() throws Exception { Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState()); } - OlapTable tbl1 = (OlapTable) db.getTable("tbl1"); + OlapTable tbl1 = db.getTableOrMetaException("tbl1", Table.TableType.OLAP); tbl1.readLock(); try { Assert.assertEquals(2, tbl1.getBaseSchema().size());