Skip to content

Commit

Permalink
[refactor] Rename Catalog to Env (apache#10702)
Browse files Browse the repository at this point in the history
  • Loading branch information
morningman authored Jul 26, 2022
1 parent 846716a commit 09224d7
Show file tree
Hide file tree
Showing 560 changed files with 4,104 additions and 4,177 deletions.
10 changes: 5 additions & 5 deletions fe/fe-core/src/main/java/org/apache/doris/PaloFe.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.doris;

import org.apache.doris.catalog.Catalog;
import org.apache.doris.catalog.Env;
import org.apache.doris.common.CommandLineOptions;
import org.apache.doris.common.Config;
import org.apache.doris.common.LdapConfig;
Expand Down Expand Up @@ -129,8 +129,8 @@ public static void start(String dorisHomeDir, String pidDir, String[] args, Star
}

// init catalog and wait it be ready
Catalog.getCurrentCatalog().initialize(args);
Catalog.getCurrentCatalog().waitForReady();
Env.getCurrentEnv().initialize(args);
Env.getCurrentEnv().waitForReady();

Telemetry.initOpenTelemetry();

Expand Down Expand Up @@ -325,7 +325,7 @@ private static void checkCommandLineOptions(CommandLineOptions cmdLineOpts) {
System.out.println("Java compile version: " + Version.DORIS_JAVA_COMPILE_VERSION);
System.exit(0);
} else if (cmdLineOpts.runBdbTools()) {
BDBTool bdbTool = new BDBTool(Catalog.getCurrentCatalog().getBdbDir(), cmdLineOpts.getBdbToolOpts());
BDBTool bdbTool = new BDBTool(Env.getCurrentEnv().getBdbDir(), cmdLineOpts.getBdbToolOpts());
if (bdbTool.run()) {
System.exit(0);
} else {
Expand All @@ -340,7 +340,7 @@ private static void checkCommandLineOptions(CommandLineOptions cmdLineOpts) {
} else {
System.out.println("Start to load image: ");
try {
MetaReader.read(imageFile, Catalog.getCurrentCatalog());
MetaReader.read(imageFile, Env.getCurrentEnv());
System.out.println("Load image success. Image file " + cmdLineOpts.getImagePath() + " is valid");
} catch (Exception e) {
System.out.println("Load image failed. Image file " + cmdLineOpts.getImagePath() + " is invalid");
Expand Down
65 changes: 32 additions & 33 deletions fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
Original file line number Diff line number Diff line change
Expand Up @@ -38,10 +38,10 @@
import org.apache.doris.analysis.RollupRenameClause;
import org.apache.doris.analysis.TableName;
import org.apache.doris.analysis.TableRenameClause;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.DataProperty;
import org.apache.doris.catalog.Database;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.MysqlTable;
import org.apache.doris.catalog.OdbcTable;
import org.apache.doris.catalog.OlapTable;
Expand Down Expand Up @@ -106,9 +106,9 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt)
String tableName = stmt.getBaseIndexName();
// check db
String dbName = stmt.getDBName();
Database db = Catalog.getCurrentInternalCatalog().getDbOrDdlException(dbName);
Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName);
// check cluster capacity
Catalog.getCurrentSystemInfo().checkClusterCapacity(stmt.getClusterName());
Env.getCurrentSystemInfo().checkClusterCapacity(stmt.getClusterName());
// check db quota
db.checkQuota();

Expand All @@ -119,7 +119,7 @@ public void processCreateMaterializedView(CreateMaterializedViewStmt stmt)
public void processDropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException {
// check db
String dbName = stmt.getTableName().getDb();
Database db = Catalog.getCurrentInternalCatalog().getDbOrDdlException(dbName);
Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName);

String tableName = stmt.getTableName().getTbl();
OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP);
Expand All @@ -142,7 +142,7 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable,

// check cluster capacity and db quota, only need to check once.
if (currentAlterOps.needCheckCapacity()) {
Catalog.getCurrentSystemInfo().checkClusterCapacity(clusterName);
Env.getCurrentSystemInfo().checkClusterCapacity(clusterName);
db.checkQuota();
}

Expand All @@ -160,7 +160,7 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable,
}
String currentStoragePolicy = currentAlterOps.getTableStoragePolicy(alterClauses);
// check currentStoragePolicy resource exist.
Catalog.getCurrentCatalog().getPolicyMgr().checkStoragePolicyExist(currentStoragePolicy);
Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(currentStoragePolicy);

olapTable.setStoragePolicy(currentStoragePolicy);
needProcessOutsideTableLock = true;
Expand All @@ -178,10 +178,9 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable,
if (!((DropPartitionClause) alterClause).isTempPartition()) {
DynamicPartitionUtil.checkAlterAllowed(olapTable);
}
Catalog.getCurrentCatalog().dropPartition(db, olapTable, ((DropPartitionClause) alterClause));
Env.getCurrentEnv().dropPartition(db, olapTable, ((DropPartitionClause) alterClause));
} else if (alterClause instanceof ReplacePartitionClause) {
Catalog.getCurrentCatalog().replaceTempPartition(
db, olapTable, (ReplacePartitionClause) alterClause);
Env.getCurrentEnv().replaceTempPartition(db, olapTable, (ReplacePartitionClause) alterClause);
} else if (alterClause instanceof ModifyPartitionClause) {
ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause);
// expand the partition names if it is 'Modify Partition(*)'
Expand Down Expand Up @@ -220,8 +219,8 @@ private boolean processAlterOlapTable(AlterTableStmt stmt, OlapTable olapTable,
} else if (currentAlterOps.contains(AlterOpType.MODIFY_DISTRIBUTION)) {
Preconditions.checkState(alterClauses.size() == 1);
AlterClause alterClause = alterClauses.get(0);
Catalog.getCurrentCatalog().modifyDefaultDistributionBucketNum(
db, olapTable, (ModifyDistributionClause) alterClause);
Env.getCurrentEnv()
.modifyDefaultDistributionBucketNum(db, olapTable, (ModifyDistributionClause) alterClause);
} else if (currentAlterOps.contains(AlterOpType.MODIFY_COLUMN_COMMENT)) {
processModifyColumnComment(db, olapTable, alterClauses);
} else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_COMMENT)) {
Expand All @@ -244,7 +243,7 @@ private void processModifyTableComment(Database db, OlapTable tbl, AlterClause a
// log
ModifyCommentOperationLog op = ModifyCommentOperationLog
.forTable(db.getId(), tbl.getId(), clause.getComment());
Catalog.getCurrentCatalog().getEditLog().logModifyComment(op);
Env.getCurrentEnv().getEditLog().logModifyComment(op);
} finally {
tbl.writeUnlock();
}
Expand Down Expand Up @@ -277,7 +276,7 @@ private void processModifyColumnComment(Database db, OlapTable tbl, List<AlterCl

// log
ModifyCommentOperationLog op = ModifyCommentOperationLog.forColumn(db.getId(), tbl.getId(), colToComment);
Catalog.getCurrentCatalog().getEditLog().logModifyComment(op);
Env.getCurrentEnv().getEditLog().logModifyComment(op);
} finally {
tbl.writeUnlock();
}
Expand All @@ -286,7 +285,7 @@ private void processModifyColumnComment(Database db, OlapTable tbl, List<AlterCl
public void replayModifyComment(ModifyCommentOperationLog operation) throws MetaNotFoundException {
long dbId = operation.getDbId();
long tblId = operation.getTblId();
Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(dbId);
Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbId);
Table tbl = db.getTableOrMetaException(tblId);
tbl.writeLock();
try {
Expand Down Expand Up @@ -338,7 +337,7 @@ public void processModifyEngine(Database db, Table externalTable, ModifyEngineCl
}

public void replayProcessModifyEngine(ModifyTableEngineOperationLog log) {
Database db = Catalog.getCurrentInternalCatalog().getDbNullable(log.getDbId());
Database db = Env.getCurrentInternalCatalog().getDbNullable(log.getDbId());
if (db == null) {
return;
}
Expand Down Expand Up @@ -381,7 +380,7 @@ private void processModifyEngineInternal(Database db, Table externalTable,
if (!isReplay) {
ModifyTableEngineOperationLog log = new ModifyTableEngineOperationLog(db.getId(),
externalTable.getId(), prop);
Catalog.getCurrentCatalog().getEditLog().logModifyTableEngine(log);
Env.getCurrentEnv().getEditLog().logModifyTableEngine(log);
}
} finally {
odbcTable.writeUnlock();
Expand All @@ -394,7 +393,7 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException {
String tableName = dbTableName.getTbl();
final String clusterName = stmt.getClusterName();

Database db = Catalog.getCurrentInternalCatalog().getDbOrDdlException(dbName);
Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName);
Table table = db.getTableOrDdlException(tableName);
List<AlterClause> alterClauses = Lists.newArrayList();
// some operations will take long time to process, need to be done outside the table lock
Expand Down Expand Up @@ -423,7 +422,7 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException {
DynamicPartitionUtil.checkAlterAllowed(
(OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP));
}
Catalog.getCurrentCatalog().addPartition(db, tableName, (AddPartitionClause) alterClause);
Env.getCurrentEnv().addPartition(db, tableName, (AddPartitionClause) alterClause);
} else if (alterClause instanceof ModifyPartitionClause) {
ModifyPartitionClause clause = ((ModifyPartitionClause) alterClause);
Map<String, String> properties = clause.getProperties();
Expand Down Expand Up @@ -476,7 +475,7 @@ private void processReplaceTable(Database db, OlapTable origTable, List<AlterCla
// write edit log
ReplaceTableOperationLog log = new ReplaceTableOperationLog(db.getId(),
origTable.getId(), olapNewTbl.getId(), swapTable);
Catalog.getCurrentCatalog().getEditLog().logReplaceTable(log);
Env.getCurrentEnv().getEditLog().logReplaceTable(log);
LOG.info("finish replacing table {} with table {}, is swap: {}", oldTblName, newTblName, swapTable);
} finally {
MetaLockUtils.writeUnlockTables(tableList);
Expand All @@ -491,7 +490,7 @@ public void replayReplaceTable(ReplaceTableOperationLog log) throws MetaNotFound
long origTblId = log.getOrigTblId();
long newTblId = log.getNewTblId();

Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(dbId);
Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbId);
OlapTable origTable = (OlapTable) db.getTableOrMetaException(origTblId, TableType.OLAP);
OlapTable newTbl = (OlapTable) db.getTableOrMetaException(newTblId, TableType.OLAP);
List<Table> tableList = Lists.newArrayList(origTable, newTbl);
Expand Down Expand Up @@ -541,15 +540,15 @@ private void replaceTableInternal(Database db, OlapTable origTable, OlapTable ne
db.createTable(origTable);
} else {
// not swap, the origin table is not used anymore, need to drop all its tablets.
Catalog.getCurrentCatalog().onEraseOlapTable(origTable, isReplay);
Env.getCurrentEnv().onEraseOlapTable(origTable, isReplay);
}
}

public void processAlterView(AlterViewStmt stmt, ConnectContext ctx) throws UserException {
TableName dbTableName = stmt.getTbl();
String dbName = dbTableName.getDb();

Database db = Catalog.getCurrentInternalCatalog().getDbOrDdlException(dbName);
Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(dbName);

String tableName = dbTableName.getTbl();
View view = (View) db.getTableOrMetaException(tableName, TableType.VIEW);
Expand All @@ -575,7 +574,7 @@ private void modifyViewDef(Database db, View view, String inlineViewDef, long sq

AlterViewInfo alterViewInfo = new AlterViewInfo(db.getId(), view.getId(),
inlineViewDef, newFullSchema, sqlMode);
Catalog.getCurrentCatalog().getEditLog().logModifyViewDef(alterViewInfo);
Env.getCurrentEnv().getEditLog().logModifyViewDef(alterViewInfo);
LOG.info("modify view[{}] definition to {}", viewName, inlineViewDef);
} finally {
view.writeUnlock();
Expand All @@ -591,7 +590,7 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws MetaNotFound
String inlineViewDef = alterViewInfo.getInlineViewDef();
List<Column> newFullSchema = alterViewInfo.getNewFullSchema();

Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(dbId);
Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbId);
View view = (View) db.getTableOrMetaException(tableId, TableType.VIEW);

db.writeLock();
Expand Down Expand Up @@ -623,17 +622,17 @@ public void processAlterCluster(AlterSystemStmt stmt) throws UserException {
private void processRename(Database db, OlapTable table, List<AlterClause> alterClauses) throws DdlException {
for (AlterClause alterClause : alterClauses) {
if (alterClause instanceof TableRenameClause) {
Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause);
Env.getCurrentEnv().renameTable(db, table, (TableRenameClause) alterClause);
break;
} else {
if (alterClause instanceof RollupRenameClause) {
Catalog.getCurrentCatalog().renameRollup(db, table, (RollupRenameClause) alterClause);
Env.getCurrentEnv().renameRollup(db, table, (RollupRenameClause) alterClause);
break;
} else if (alterClause instanceof PartitionRenameClause) {
Catalog.getCurrentCatalog().renamePartition(db, table, (PartitionRenameClause) alterClause);
Env.getCurrentEnv().renamePartition(db, table, (PartitionRenameClause) alterClause);
break;
} else if (alterClause instanceof ColumnRenameClause) {
Catalog.getCurrentCatalog().renameColumn(db, table, (ColumnRenameClause) alterClause);
Env.getCurrentEnv().renameColumn(db, table, (ColumnRenameClause) alterClause);
break;
} else {
Preconditions.checkState(false);
Expand All @@ -645,7 +644,7 @@ private void processRename(Database db, OlapTable table, List<AlterClause> alter
private void processRename(Database db, Table table, List<AlterClause> alterClauses) throws DdlException {
for (AlterClause alterClause : alterClauses) {
if (alterClause instanceof TableRenameClause) {
Catalog.getCurrentCatalog().renameTable(db, table, (TableRenameClause) alterClause);
Env.getCurrentEnv().renameTable(db, table, (TableRenameClause) alterClause);
break;
} else {
Preconditions.checkState(false);
Expand Down Expand Up @@ -684,7 +683,7 @@ public void modifyPartitionsProperty(Database db,
// get value from properties here
// 1. replica allocation
ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
Catalog.getCurrentSystemInfo().checkReplicaAllocation(db.getClusterName(), replicaAlloc);
Env.getCurrentSystemInfo().checkReplicaAllocation(db.getClusterName(), replicaAlloc);
// 2. in memory
boolean newInMemory = PropertyAnalyzer.analyzeBooleanProp(properties,
PropertyAnalyzer.PROPERTIES_INMEMORY, false);
Expand Down Expand Up @@ -712,7 +711,7 @@ public void modifyPartitionsProperty(Database db,
String currentStoragePolicy = PropertyAnalyzer.analyzeStoragePolicy(properties);
if (!currentStoragePolicy.equals("")) {
// check currentStoragePolicy resource exist.
Catalog.getCurrentCatalog().getPolicyMgr().checkStoragePolicyExist(currentStoragePolicy);
Env.getCurrentEnv().getPolicyMgr().checkStoragePolicyExist(currentStoragePolicy);
partitionInfo.setStoragePolicy(partition.getId(), currentStoragePolicy);
}

Expand Down Expand Up @@ -743,11 +742,11 @@ public void modifyPartitionsProperty(Database db,

// log here
BatchModifyPartitionsInfo info = new BatchModifyPartitionsInfo(modifyPartitionInfos);
Catalog.getCurrentCatalog().getEditLog().logBatchModifyPartition(info);
Env.getCurrentEnv().getEditLog().logBatchModifyPartition(info);
}

public void replayModifyPartition(ModifyPartitionInfo info) throws MetaNotFoundException {
Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId());
Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId());
OlapTable olapTable = (OlapTable) db.getTableOrMetaException(info.getTableId(), TableType.OLAP);
olapTable.writeLock();
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@

import org.apache.doris.analysis.AlterClause;
import org.apache.doris.analysis.CancelStmt;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.catalog.Database;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.MaterializedIndex;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.Partition;
Expand Down Expand Up @@ -118,7 +118,7 @@ private void clearExpireFinishedOrCancelledAlterJobsV2() {
iterator.remove();
RemoveAlterJobV2OperationLog log = new RemoveAlterJobV2OperationLog(
alterJobV2.getJobId(), alterJobV2.getType());
Catalog.getCurrentCatalog().getEditLog().logRemoveExpiredAlterJobV2(log);
Env.getCurrentEnv().getEditLog().logRemoveExpiredAlterJobV2(log);
LOG.info("remove expired {} job {}. finish at {}", alterJobV2.getType(),
alterJobV2.getJobId(), TimeUtils.longToTimeString(alterJobV2.getFinishedTimeMs()));
}
Expand Down Expand Up @@ -198,7 +198,7 @@ public void processExternalTable(List<AlterClause> alterClauses, Database db, Ta
* 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.getCurrentInternalCatalog().getDbOrMetaException(task.getDbId());
Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(task.getDbId());

OlapTable tbl = (OlapTable) db.getTableOrMetaException(task.getTableId(), Table.TableType.OLAP);
tbl.writeLockOrMetaException();
Expand Down Expand Up @@ -233,7 +233,7 @@ public void handleFinishAlterTask(AlterReplicaTask task) throws MetaNotFoundExce
replica.getId(), replica.getVersion(), -1,
replica.getDataSize(), replica.getRemoteDataSize(), replica.getRowCount(),
replica.getLastFailedVersion(), replica.getLastSuccessVersion());
Catalog.getCurrentCatalog().getEditLog().logUpdateReplica(info);
Env.getCurrentEnv().getEditLog().logUpdateReplica(info);
}

LOG.info("after handle alter task tablet: {}, replica: {}", task.getSignature(), replica);
Expand Down
Loading

0 comments on commit 09224d7

Please sign in to comment.