diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java index f8084cba19130..163f70d40fe8d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java @@ -15,13 +15,16 @@ package com.starrocks.alter; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Range; import com.starrocks.analysis.DateLiteral; import com.starrocks.analysis.ParseNode; import com.starrocks.analysis.TableName; import com.starrocks.analysis.TableRef; import com.starrocks.catalog.ColocateTableIndex; +import com.starrocks.catalog.Column; import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; import com.starrocks.catalog.DynamicPartitionProperty; @@ -34,11 +37,13 @@ import com.starrocks.catalog.PartitionType; import com.starrocks.catalog.RangePartitionInfo; import com.starrocks.catalog.Table; +import com.starrocks.catalog.TableProperty; import com.starrocks.catalog.Type; import com.starrocks.common.AnalysisException; import com.starrocks.common.DdlException; import com.starrocks.common.ErrorCode; import com.starrocks.common.ErrorReport; +import com.starrocks.common.ErrorReportException; import com.starrocks.common.InvalidOlapTableStateException; import com.starrocks.common.MaterializedViewExceptions; import com.starrocks.common.UserException; @@ -50,11 +55,13 @@ import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.persist.AlterViewInfo; import com.starrocks.persist.BatchModifyPartitionsInfo; +import com.starrocks.persist.ColumnRenameInfo; import com.starrocks.persist.ModifyPartitionInfo; +import com.starrocks.persist.ModifyTablePropertyOperationLog; import com.starrocks.persist.SwapTableOperationLog; +import com.starrocks.persist.TableInfo; import com.starrocks.qe.ConnectContext; import com.starrocks.server.GlobalStateMgr; -import com.starrocks.server.LocalMetastore; import com.starrocks.sql.analyzer.SemanticException; import com.starrocks.sql.ast.AddColumnClause; import com.starrocks.sql.ast.AddColumnsClause; @@ -99,11 +106,13 @@ import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; +import static com.starrocks.meta.StarRocksMeta.inactiveRelatedMaterializedView; import static com.starrocks.sql.common.UnsupportedException.unsupportedException; public class AlterJobExecutor implements AstVisitor { @@ -270,7 +279,7 @@ public Void visitTableRenameClause(TableRenameClause clause, ConnectContext cont locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); try { ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().renameTable(db, table, clause)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().renameTable(db, table, clause)); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); } @@ -282,7 +291,7 @@ public Void visitAlterTableCommentClause(AlterTableCommentClause clause, Connect Locker locker = new Locker(); locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); try { - ErrorReport.wrapWithRuntimeException(() -> GlobalStateMgr.getCurrentState().getLocalMetastore() + ErrorReport.wrapWithRuntimeException(() -> GlobalStateMgr.getCurrentState().getStarRocksMeta() .alterTableComment(db, table, clause)); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); @@ -317,9 +326,9 @@ public Void visitSwapTableClause(SwapTableClause clause, ConnectContext context) } // inactive the related MVs - LocalMetastore.inactiveRelatedMaterializedView(db, origTable, + inactiveRelatedMaterializedView(db, origTable, MaterializedViewExceptions.inactiveReasonForBaseTableSwapped(origTblName)); - LocalMetastore.inactiveRelatedMaterializedView(db, olapNewTbl, + inactiveRelatedMaterializedView(db, olapNewTbl, MaterializedViewExceptions.inactiveReasonForBaseTableSwapped(newTblName)); SwapTableOperationLog log = new SwapTableOperationLog(db.getId(), origTable.getId(), olapNewTbl.getId()); @@ -401,7 +410,7 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, GlobalStateMgr.getCurrentState().getColocateTableIndex() .modifyTableColocate(db, olapTable, colocateGroup, false, null); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().convertDistributionType(db, olapTable); + //GlobalStateMgr.getCurrentState().getLocalMetastore().convertDistributionType(db, olapTable); } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) { if (!olapTable.dynamicPartitionExists()) { try { @@ -420,24 +429,22 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, "modify dynamic_partition.buckets. Colocate tables must have same bucket number."); } } - GlobalStateMgr.getCurrentState().getLocalMetastore() - .modifyTableDynamicPartition(db, olapTable, properties); + modifyTableDynamicPartition(db, olapTable, properties); } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { Preconditions.checkNotNull(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .modifyTableDefaultReplicationNum(db, olapTable, properties); + modifyTableDefaultReplicationNum(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().modifyTableReplicationNum(db, olapTable, properties); + modifyTableReplicationNum(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); + alterTableProperties(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); + alterTableProperties(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); + alterTableProperties(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); + alterTableProperties(db, olapTable, properties); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION)) { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); + alterTableProperties(db, olapTable, properties); } else { schemaChangeHandler.process(Lists.newArrayList(clause), db, olapTable); } @@ -453,6 +460,236 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, return null; } + public void modifyTableDynamicPartition(Database db, OlapTable table, Map properties) + throws DdlException { + Map logProperties = new HashMap<>(properties); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + DynamicPartitionUtil.checkAndSetDynamicPartitionProperty(table, properties); + } else { + Map analyzedDynamicPartition = DynamicPartitionUtil.analyzeDynamicPartition(properties); + tableProperty.modifyTableProperties(analyzedDynamicPartition); + tableProperty.buildDynamicProperty(); + } + + DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), table); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), logProperties); + GlobalStateMgr.getCurrentState().getEditLog().logDynamicPartition(info); + } + + /** + * Set replication number for unpartitioned table. + * ATTN: only for unpartitioned table now. + * + * @param db + * @param table + * @param properties + * @throws DdlException + */ + // The caller need to hold the db write lock + public void modifyTableReplicationNum(Database db, OlapTable table, Map properties) + throws DdlException { + if (GlobalStateMgr.getCurrentState().getColocateTableIndex().isColocateTable(table.getId())) { + throw new DdlException("table " + table.getName() + " is colocate table, cannot change replicationNum"); + } + + String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM; + PartitionInfo partitionInfo = table.getPartitionInfo(); + if (partitionInfo.isRangePartition()) { + throw new DdlException( + "This is a range partitioned table, you should specify partitions with MODIFY PARTITION clause." + + " If you want to set default replication number, please use '" + defaultReplicationNumName + + "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM + "' to escape misleading."); + } + + // unpartitioned table + // update partition replication num + String partitionName = table.getName(); + Partition partition = table.getPartition(partitionName); + if (partition == null) { + throw new DdlException("Partition does not exist. name: " + partitionName); + } + + short replicationNum = Short.parseShort(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); + boolean isInMemory = partitionInfo.getIsInMemory(partition.getId()); + DataProperty newDataProperty = partitionInfo.getDataProperty(partition.getId()); + partitionInfo.setReplicationNum(partition.getId(), replicationNum); + + // update table default replication num + table.setReplicationNum(replicationNum); + + // log + ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(), + newDataProperty, replicationNum, isInMemory); + GlobalStateMgr.getCurrentState().getEditLog().logModifyPartition(info); + LOG.info("modify partition[{}-{}-{}] replication num to {}", db.getOriginName(), table.getName(), + partition.getName(), replicationNum); + } + + /** + * Set default replication number for a specified table. + * You can see the default replication number by Show Create Table stmt. + * + * @param db + * @param table + * @param properties + */ + // The caller need to hold the db write lock + public void modifyTableDefaultReplicationNum(Database db, OlapTable table, Map properties) + throws DdlException { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + if (GlobalStateMgr.getCurrentState().getColocateTableIndex().isColocateTable(table.getId())) { + throw new DdlException("table " + table.getName() + " is colocate table, cannot change replicationNum"); + } + + // check unpartitioned table + PartitionInfo partitionInfo = table.getPartitionInfo(); + Partition partition = null; + boolean isUnpartitionedTable = false; + if (partitionInfo.getType() == PartitionType.UNPARTITIONED) { + isUnpartitionedTable = true; + String partitionName = table.getName(); + partition = table.getPartition(partitionName); + if (partition == null) { + throw new DdlException("Partition does not exist. name: " + partitionName); + } + } + + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildReplicationNum(); + + // update partition replication num if this table is unpartitioned table + if (isUnpartitionedTable) { + Preconditions.checkNotNull(partition); + partitionInfo.setReplicationNum(partition.getId(), tableProperty.getReplicationNum()); + } + + // log + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyReplicationNum(info); + LOG.info("modify table[{}] replication num to {}", table.getName(), + properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); + } + + public void alterTableProperties(Database db, OlapTable table, Map properties) + throws DdlException { + Map propertiesToPersist = new HashMap<>(properties); + Map results = validateToBeModifiedProps(properties, table); + + TableProperty tableProperty = table.getTableProperty(); + for (String key : results.keySet()) { + if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER)) { + int partitionLiveNumber = (int) results.get(key); + tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER, + String.valueOf(partitionLiveNumber)); + if (partitionLiveNumber == TableProperty.INVALID) { + GlobalStateMgr.getCurrentState().getDynamicPartitionScheduler().removeTtlPartitionTable(db.getId(), + table.getId()); + } else { + GlobalStateMgr.getCurrentState().getDynamicPartitionScheduler().registerTtlPartitionTable(db.getId(), + table.getId()); + } + tableProperty.setPartitionTTLNumber(partitionLiveNumber); + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + ImmutableMap.of(key, propertiesToPersist.get(key))); + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); + } + if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM)) { + DataProperty dataProperty = (DataProperty) results.get(key); + TStorageMedium storageMedium = dataProperty.getStorageMedium(); + table.setStorageMedium(storageMedium); + tableProperty.getProperties() + .put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TIME, + String.valueOf(dataProperty.getCooldownTimeMs())); + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + ImmutableMap.of(key, propertiesToPersist.get(key))); + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); + } + if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL)) { + String storageCoolDownTTL = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL); + tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL, storageCoolDownTTL); + tableProperty.buildStorageCoolDownTTL(); + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + ImmutableMap.of(key, propertiesToPersist.get(key))); + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); + } + if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION)) { + String partitionDuration = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION); + tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION, partitionDuration); + tableProperty.buildDataCachePartitionDuration(); + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + ImmutableMap.of(key, propertiesToPersist.get(key))); + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); + } + if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION)) { + String location = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION); + table.setLocation(location); + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + ImmutableMap.of(key, propertiesToPersist.get(key))); + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); + } + } + } + + private Map validateToBeModifiedProps(Map properties, OlapTable table) throws DdlException { + Map results = Maps.newHashMap(); + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER)) { + int partitionLiveNumber = PropertyAnalyzer.analyzePartitionLiveNumber(properties, true); + results.put(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER, partitionLiveNumber); + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM)) { + try { + DataProperty dataProperty = DataProperty.getInferredDefaultDataProperty(); + dataProperty = PropertyAnalyzer.analyzeDataProperty(properties, dataProperty, false); + results.put(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM, dataProperty); + } catch (AnalysisException ex) { + throw new RuntimeException(ex.getMessage()); + } + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL)) { + try { + PropertyAnalyzer.analyzeStorageCoolDownTTL(properties, true); + results.put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL, null); + } catch (AnalysisException ex) { + throw new RuntimeException(ex.getMessage()); + } + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION)) { + try { + PropertyAnalyzer.analyzeDataCachePartitionDuration(properties); + results.put(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION, null); + } catch (AnalysisException ex) { + throw new RuntimeException(ex.getMessage()); + } + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION)) { + if (table.getColocateGroup() != null) { + throw new DdlException("Cannot set location for colocate table"); + } + String locations = PropertyAnalyzer.analyzeLocation(properties, true); + results.put(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION, locations); + } + if (!properties.isEmpty()) { + throw new DdlException("Modify failed because unknown properties: " + properties); + } + return results; + } + @Override public Void visitOptimizeClause(OptimizeClause clause, ConnectContext context) { unsupportedException("Not support"); @@ -493,13 +730,43 @@ public Void visitColumnRenameClause(ColumnRenameClause clause, ConnectContext co modifiedColumns.add(clause.getColName()); ErrorReport.wrapWithRuntimeException(() -> schemaChangeHandler.checkModifiedColumWithMaterializedViews((OlapTable) table, modifiedColumns)); - GlobalStateMgr.getCurrentState().getLocalMetastore().renameColumn(db, table, clause); + renameColumn(db, table, clause); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); } return null; } + public void renameColumn(Database db, Table table, ColumnRenameClause renameClause) { + if (!(table instanceof OlapTable)) { + throw ErrorReportException.report(ErrorCode.ERR_COLUMN_RENAME_ONLY_FOR_OLAP_TABLE); + } + if (db.isSystemDatabase() || db.isStatisticsDatabase()) { + throw ErrorReportException.report(ErrorCode.ERR_CANNOT_RENAME_COLUMN_IN_INTERNAL_DB, db.getFullName()); + } + OlapTable olapTable = (OlapTable) table; + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw ErrorReportException.report(ErrorCode.ERR_CANNOT_RENAME_COLUMN_OF_NOT_NORMAL_TABLE, olapTable.getState()); + } + + String colName = renameClause.getColName(); + String newColName = renameClause.getNewColName(); + + Column column = olapTable.getColumn(colName); + if (column == null) { + throw ErrorReportException.report(ErrorCode.ERR_BAD_FIELD_ERROR, colName, table.getName()); + } + Column currentColumn = olapTable.getColumn(newColName); + if (currentColumn != null) { + throw ErrorReportException.report(ErrorCode.ERR_DUP_FIELDNAME, newColName); + } + olapTable.renameColumn(colName, newColName); + + ColumnRenameInfo columnRenameInfo = new ColumnRenameInfo(db.getId(), table.getId(), colName, newColName); + GlobalStateMgr.getCurrentState().getEditLog().logColumnRename(columnRenameInfo); + LOG.info("rename column {} to {}", colName, newColName); + } + @Override public Void visitReorderColumnsClause(ReorderColumnsClause clause, ConnectContext context) { unsupportedException("Not support"); @@ -523,14 +790,48 @@ public Void visitRollupRenameClause(RollupRenameClause clause, ConnectContext co Locker locker = new Locker(); locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); try { - ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().renameRollup(db, (OlapTable) table, clause)); + ErrorReport.wrapWithRuntimeException(() -> renameRollup(db, (OlapTable) table, clause)); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); } return null; } + public void renameRollup(Database db, OlapTable table, RollupRenameClause renameClause) throws DdlException { + if (table.getState() != OlapTable.OlapTableState.NORMAL) { + throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); + } + + String rollupName = renameClause.getRollupName(); + // check if it is base table name + if (rollupName.equals(table.getName())) { + throw new DdlException("Using ALTER TABLE RENAME to change table name"); + } + + String newRollupName = renameClause.getNewRollupName(); + if (rollupName.equals(newRollupName)) { + throw new DdlException("Same rollup name"); + } + + Map indexNameToIdMap = table.getIndexNameToId(); + if (indexNameToIdMap.get(rollupName) == null) { + throw new DdlException("Rollup index[" + rollupName + "] does not exists"); + } + + // check if name is already used + if (indexNameToIdMap.get(newRollupName) != null) { + throw new DdlException("Rollup name[" + newRollupName + "] is already used"); + } + + long indexId = indexNameToIdMap.remove(rollupName); + indexNameToIdMap.put(newRollupName, indexId); + + // log + TableInfo tableInfo = TableInfo.createForRollupRename(db.getId(), table.getId(), indexId, newRollupName); + GlobalStateMgr.getCurrentState().getEditLog().logRollupRename(tableInfo); + LOG.info("rename rollup[{}] to {}", rollupName, newRollupName); + } + @Override public Void visitCompactionClause(CompactionClause clause, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> @@ -564,7 +865,7 @@ public Void visitAddPartitionClause(AddPartitionClause clause, ConnectContext co DynamicPartitionUtil.checkAlterAllowed((OlapTable) table); } ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().addPartitions(context, db, table.getName(), clause)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addPartitions(context, db, table.getName(), clause)); return null; } @@ -581,7 +882,7 @@ public Void visitDropPartitionClause(DropPartitionClause clause, ConnectContext } ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(db, table, clause)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropPartition(db, table, clause)); return null; } @@ -595,14 +896,14 @@ public Void visitTruncatePartitionClause(TruncatePartitionClause clause, Connect ctx.setGlobalStateMgr(GlobalStateMgr.getCurrentState()); ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().truncateTable(tStmt, ctx)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().truncateTable(tStmt, ctx)); return null; } @Override public Void visitReplacePartitionClause(ReplacePartitionClause clause, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().replaceTempPartition(db, table.getName(), clause)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().replaceTempPartition(db, table.getName(), clause)); return null; } @@ -616,7 +917,7 @@ public Void visitPartitionRenameClause(PartitionRenameClause clause, ConnectCont } ErrorReport.wrapWithRuntimeException(() -> - GlobalStateMgr.getCurrentState().getLocalMetastore().renamePartition(db, table, clause)); + GlobalStateMgr.getCurrentState().getStarRocksMeta().renamePartition(db, table, clause)); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.WRITE); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java index 3e9922bf5a0e7..e8bcd71c5d9ea 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java @@ -80,14 +80,15 @@ import com.starrocks.scheduler.Task; import com.starrocks.scheduler.TaskBuilder; import com.starrocks.server.GlobalStateMgr; -import com.starrocks.server.LocalMetastore; import com.starrocks.sql.analyzer.Analyzer; import com.starrocks.sql.analyzer.MaterializedViewAnalyzer; import com.starrocks.sql.analyzer.SemanticException; import com.starrocks.sql.ast.AlterMaterializedViewStatusClause; +import com.starrocks.sql.ast.CancelAlterTableStmt; import com.starrocks.sql.ast.CreateMaterializedViewStatement; import com.starrocks.sql.ast.DropMaterializedViewStmt; import com.starrocks.sql.ast.QueryStatement; +import com.starrocks.sql.ast.ShowAlterStmt; import com.starrocks.sql.ast.StatementBase; import com.starrocks.sql.ast.UserIdentity; import com.starrocks.sql.optimizer.rule.transformation.materialization.MvUtils; @@ -102,6 +103,8 @@ import java.util.Optional; import java.util.stream.Collectors; +import static com.starrocks.meta.StarRocksMeta.inactiveRelatedMaterializedView; + public class AlterJobMgr { private static final Logger LOG = LogManager.getLogger(AlterJobMgr.class); public static final String MANUAL_INACTIVE_MV_REASON = "user use alter materialized view set status to inactive"; @@ -231,7 +234,7 @@ public static QueryStatement recreateMVQuery(MaterializedView materializedView, String createMvSql) { // If we could parse the MV sql successfully, and the schema of mv does not change, // we could reuse the existing MV - Optional mayDb = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(materializedView.getDbId()); + Optional mayDb = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(materializedView.getDbId()); // check database existing String dbName = mayDb.orElseThrow(() -> @@ -498,7 +501,7 @@ public void alterView(AlterViewInfo alterViewInfo) { } view.setNewFullSchema(newFullSchema); view.setComment(comment); - LocalMetastore.inactiveRelatedMaterializedView(db, view, + inactiveRelatedMaterializedView(db, view, MaterializedViewExceptions.inactiveReasonForBaseViewChanged(viewName)); db.dropTable(viewName); db.registerTableUnlocked(view); @@ -535,6 +538,23 @@ public void replayModifyPartition(ModifyPartitionInfo info) { } } + /* + * used for handling CancelAlterStmt (for client is the CANCEL ALTER + * command). including SchemaChangeHandler and RollupHandler + */ + public void cancelAlter(CancelAlterTableStmt stmt, String reason) throws DdlException { + if (stmt.getAlterType() == ShowAlterStmt.AlterType.ROLLUP) { + materializedViewHandler.cancel(stmt, reason); + } else if (stmt.getAlterType() == ShowAlterStmt.AlterType.COLUMN + || stmt.getAlterType() == ShowAlterStmt.AlterType.OPTIMIZE) { + schemaChangeHandler.cancel(stmt, reason); + } else if (stmt.getAlterType() == ShowAlterStmt.AlterType.MATERIALIZED_VIEW) { + materializedViewHandler.cancelMV(stmt); + } else { + throw new DdlException("Cancel " + stmt.getAlterType() + " does not implement yet"); + } + } + public SchemaChangeHandler getSchemaChangeHandler() { return this.schemaChangeHandler; } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/AlterMVJobExecutor.java b/fe/fe-core/src/main/java/com/starrocks/alter/AlterMVJobExecutor.java index fa9e6f2b6d70f..86a262be23fc5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/AlterMVJobExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/AlterMVJobExecutor.java @@ -407,7 +407,7 @@ public Void visitAlterMaterializedViewStatusClause(AlterMaterializedViewStatusCl alterMaterializedViewStatus(materializedView, status, false); // for manual refresh type, do not refresh if (materializedView.getRefreshScheme().getType() != MaterializedView.RefreshType.MANUAL) { - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .refreshMaterializedView(dbName, materializedView.getName(), true, null, Constants.TaskRunPriority.NORMAL.value(), true, false); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/CompactionHandler.java b/fe/fe-core/src/main/java/com/starrocks/alter/CompactionHandler.java index 64ec32d45919a..48c765de16d3b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/CompactionHandler.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/CompactionHandler.java @@ -28,6 +28,7 @@ import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.lake.compaction.CompactionMgr; import com.starrocks.lake.compaction.PartitionIdentifier; +import com.starrocks.meta.TabletMetastore; import com.starrocks.qe.ShowResultSet; import com.starrocks.server.GlobalStateMgr; import com.starrocks.server.RunMode; @@ -47,7 +48,7 @@ import java.util.ArrayList; import java.util.List; -public class CompactionHandler { +public class CompactionHandler { private static final Logger LOG = LogManager.getLogger(CompactionHandler.class); // add synchronized to avoid process 2 or more stmts at same time @@ -82,11 +83,16 @@ public static synchronized ShowResultSet process(List alterClauses, locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(olapTable.getId()), LockType.READ); try { List allPartitions = findAllPartitions(olapTable, compactionClause); + + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); for (Partition partition : allPartitions) { - for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { - for (MaterializedIndex index : physicalPartition.getMaterializedIndices( - MaterializedIndex.IndexExtState.VISIBLE)) { - for (Tablet tablet : index.getTablets()) { + List physicalPartitionList = tabletMetastore.getAllPhysicalPartition(partition); + for (PhysicalPartition physicalPartition : physicalPartitionList) { + List materializedIndices = tabletMetastore + .getMaterializedIndices(physicalPartition, MaterializedIndex.IndexExtState.VISIBLE); + for (MaterializedIndex materializedIndex : materializedIndices) { + List tabletList = tabletMetastore.getAllTablets(materializedIndex); + for (Tablet tablet : tabletList) { for (Long backendId : ((LocalTablet) tablet).getBackendIds()) { backendToTablets.put(backendId, tablet.getId()); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java index fecad154bd0c4..d332920417069 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterJobV2Builder.java @@ -75,7 +75,11 @@ public AlterJobV2 build() throws UserException { long partitionId = partition.getParentId(); long physicalPartitionId = partition.getId(); long shardGroupId = partition.getShardGroupId(); - List originTablets = partition.getIndex(originIndexId).getTablets(); + + MaterializedIndex materializedIndex = partition.getIndex(originIndexId); + List originTablets = GlobalStateMgr.getCurrentState().getTabletMetastore() + .getAllTablets(materializedIndex); + // TODO: It is not good enough to create shards into the same group id, schema change PR needs to // revise the code again. List originTabletIds = originTablets.stream().map(Tablet::getId).collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java index 8eb3a5e0a4b7c..eee2dfadbe682 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableAlterMetaJobBase.java @@ -263,7 +263,8 @@ boolean publishVersion() { long commitVersion = commitVersionMap.get(partitionId); Map dirtyIndexMap = physicalPartitionIndexMap.row(partitionId); for (MaterializedIndex index : dirtyIndexMap.values()) { - Utils.publishVersion(index.getTablets(), txnInfo, commitVersion - 1, commitVersion, + List tabletList = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index); + Utils.publishVersion(tabletList, txnInfo, commitVersion - 1, commitVersion, warehouseId); } } @@ -319,7 +320,7 @@ public void updateIndexTabletMeta(Database db, OlapTable table, PhysicalPartitio Locker locker = new Locker(); locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.READ); try { - tablets = new ArrayList<>(index.getTablets()); + tablets = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(table.getId()), LockType.READ); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java index 72e11235cf4e2..2f898fd0b066d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/LakeTableSchemaChangeJob.java @@ -90,7 +90,6 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; @@ -338,7 +337,9 @@ protected void runPendingJob() throws AlterCancelException { if (enableTabletCreationOptimization) { numTablets = physicalPartitionIndexMap.size(); } else { - numTablets = physicalPartitionIndexMap.values().stream().map(MaterializedIndex::getTablets) + numTablets = physicalPartitionIndexMap.values().stream() + .map(materializedIndex -> GlobalStateMgr.getCurrentState().getTabletMetastore() + .getAllTablets(materializedIndex)) .mapToLong(List::size).sum(); } countDownLatch = new MarkedCountDownLatch<>((int) numTablets); @@ -373,7 +374,7 @@ protected void runPendingJob() throws AlterCancelException { .build().toTabletSchema(); boolean createSchemaFile = true; - for (Tablet shadowTablet : shadowIdx.getTablets()) { + for (Tablet shadowTablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(shadowIdx)) { long shadowTabletId = shadowTablet.getId(); ComputeNode computeNode = GlobalStateMgr.getCurrentState().getWarehouseMgr() .getComputeNodeAssignedToTablet(warehouseId, (LakeTablet) shadowTablet); @@ -418,7 +419,7 @@ protected void runPendingJob() throws AlterCancelException { } sendAgentTaskAndWait(batchTask, countDownLatch, Config.tablet_create_timeout_second * numTablets, - waitingCreatingReplica, isCancelling); + waitingCreatingReplica, isCancelling); // Add shadow indexes to table. try (WriteLockedDatabase db = getWriteLockedDatabase(dbId)) { @@ -483,7 +484,7 @@ protected void runWaitingTxnJob() throws AlterCancelException { for (Map.Entry entry : shadowIndexMap.entrySet()) { long shadowIdxId = entry.getKey(); MaterializedIndex shadowIdx = entry.getValue(); - for (Tablet shadowTablet : shadowIdx.getTablets()) { + for (Tablet shadowTablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(shadowIdx)) { ComputeNode computeNode = GlobalStateMgr.getCurrentState().getWarehouseMgr() .getComputeNodeAssignedToTablet(warehouseId, (LakeTablet) shadowTablet); if (computeNode == null) { @@ -610,7 +611,7 @@ protected void runFinishedRewritingJob() throws AlterCancelException { // Delete tablet and shards for (MaterializedIndex droppedIndex : droppedIndexes) { - List shards = droppedIndex.getTablets().stream().map(Tablet::getId).collect(Collectors.toList()); + List shards = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTabletIDs(droppedIndex); // TODO: what if unusedShards deletion is partially successful? StarMgrMetaSyncer.dropTabletAndDeleteShard(shards, GlobalStateMgr.getCurrentState().getStarOSAgent()); } @@ -668,7 +669,9 @@ boolean publishVersion() { long commitVersion = commitVersionMap.get(partitionId); Map shadowIndexMap = physicalPartitionIndexMap.row(partitionId); for (MaterializedIndex shadowIndex : shadowIndexMap.values()) { - Utils.publishVersion(shadowIndex.getTablets(), txnInfo, 1, commitVersion, warehouseId); + Utils.publishVersion( + GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(shadowIndex), + txnInfo, 1, commitVersion, warehouseId); } } return true; @@ -700,7 +703,7 @@ private void inactiveRelatedMv(Set modifiedColumns, @NotNull OlapTable t Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); for (MvId mvId : tbl.getRelatedMaterializedViews()) { MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore() - .getTable(db.getId(), mvId.getId()); + .getTable(db.getId(), mvId.getId()); if (mv == null) { LOG.warn("Ignore materialized view {} does not exists", mvId); continue; diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/com/starrocks/alter/MaterializedViewHandler.java index d828f8db9fd4c..84d287075b3c2 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/MaterializedViewHandler.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/MaterializedViewHandler.java @@ -285,7 +285,7 @@ public void processBatchAddRollup(List alterClauses, Database db, O TabletInvertedIndex tabletInvertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); for (RollupJobV2 rollupJobV2 : rollupNameJobMap.values()) { for (MaterializedIndex index : rollupJobV2.getPartitionIdToRollupIndex().values()) { - for (Tablet tablet : index.getTablets()) { + for (Tablet tablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index)) { tabletInvertedIndex.deleteTablet(tablet.getId()); } } @@ -459,7 +459,7 @@ private List checkAndPrepareMaterializedView(CreateMaterializedViewStmt // check if mv index already exists in db - if (GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetTable(db.getFullName(), mvName).isPresent()) { + if (GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetTable(db.getFullName(), mvName).isPresent()) { throw new DdlException("Table [" + mvName + "] already exists in the db " + db.getFullName()); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/OnlineOptimizeJobV2.java b/fe/fe-core/src/main/java/com/starrocks/alter/OnlineOptimizeJobV2.java index 5d0ed124e5e76..6b641e4d1b0a5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/OnlineOptimizeJobV2.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/OnlineOptimizeJobV2.java @@ -550,7 +550,7 @@ private void cancelInternal() { if (partition != null) { for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { // hash set is able to deduplicate the elements - tmpTablets.addAll(index.getTablets()); + tmpTablets.addAll(GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index)); } targetTable.dropTempPartition(partition.getName(), true); } else { diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeHandler.java index 79947502b66a9..ce5d8d0aa3b3f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeHandler.java @@ -88,6 +88,9 @@ import com.starrocks.common.util.concurrent.MarkedCountDownLatch; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.meta.TabletMetastore; +import com.starrocks.persist.ModifyTablePropertyOperationLog; +import com.starrocks.persist.OperationType; import com.starrocks.persist.TableAddOrDropColumnsInfo; import com.starrocks.qe.ConnectContext; import com.starrocks.qe.ShowResultSet; @@ -2124,12 +2127,189 @@ public void updateTableMeta(Database db, String tableName, Map p locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(olapTable.getId()), LockType.WRITE); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().modifyTableMeta(db, olapTable, properties, metaType); + modifyTableMeta(db, olapTable, properties, metaType); } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(olapTable.getId()), LockType.WRITE); } } + public void modifyTableMeta(Database db, OlapTable table, Map properties, + TTabletMetaType metaType) { + if (metaType == TTabletMetaType.INMEMORY) { + modifyTableInMemoryMeta(db, table, properties); + } else if (metaType == TTabletMetaType.ENABLE_PERSISTENT_INDEX) { + modifyTableEnablePersistentIndexMeta(db, table, properties); + } else if (metaType == TTabletMetaType.WRITE_QUORUM) { + modifyTableWriteQuorum(db, table, properties); + } else if (metaType == TTabletMetaType.REPLICATED_STORAGE) { + modifyTableReplicatedStorage(db, table, properties); + } else if (metaType == TTabletMetaType.BUCKET_SIZE) { + modifyTableAutomaticBucketSize(db, table, properties); + } else if (metaType == TTabletMetaType.MUTABLE_BUCKET_NUM) { + modifyTableMutableBucketNum(db, table, properties); + } else if (metaType == TTabletMetaType.PRIMARY_INDEX_CACHE_EXPIRE_SEC) { + modifyTablePrimaryIndexCacheExpireSec(db, table, properties); + } else if (metaType == TTabletMetaType.ENABLE_LOAD_PROFILE) { + modifyTableEnableLoadProfile(db, table, properties); + } + } + + // The caller need to hold the db write lock + private void modifyTableInMemoryMeta(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildInMemory(); + + // need to update partition info meta + for (Partition partition : table.getPartitions()) { + table.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory()); + } + + GlobalStateMgr.getCurrentState().getLocalMetastore().modifyTableProperty( + db, table, properties, OperationType.OP_MODIFY_IN_MEMORY); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyInMemory(info); + } + + private void modifyTableEnablePersistentIndexMeta(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildEnablePersistentIndex(); + + if (table.isCloudNativeTable()) { + // now default to LOCAL + tableProperty.buildPersistentIndexType(); + } + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyEnablePersistentIndex(info); + + } + + // The caller need to hold the db write lock + private void modifyTableWriteQuorum(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildWriteQuorum(); + + GlobalStateMgr.getCurrentState().getLocalMetastore().modifyTableProperty(db, table, properties); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyWriteQuorum(info); + } + + // The caller need to hold the db write lock + private void modifyTableReplicatedStorage(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildReplicatedStorage(); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyReplicatedStorage(info); + } + + // The caller need to hold the db write lock + private void modifyTableAutomaticBucketSize(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildBucketSize(); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyBucketSize(info); + } + + private void modifyTableMutableBucketNum(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildMutableBucketNum(); + + ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyMutableBucketNum(info); + } + + private void modifyTableEnableLoadProfile(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildEnableLoadProfile(); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyEnableLoadProfile(info); + } + + private void modifyTablePrimaryIndexCacheExpireSec(Database db, OlapTable table, Map properties) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + TableProperty tableProperty = table.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + table.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildPrimaryIndexCacheExpireSec(); + + ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), + properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyPrimaryIndexCacheExpireSec(info); + } + // return true means that the modification of FEMeta is successful, // and as long as the modification of metadata is successful, // the final consistency will be achieved through the report handler @@ -2203,7 +2383,7 @@ public boolean updateBinlogConfigMeta(Database db, Long tableId, Map tabletSet = beIdToTabletId.computeIfAbsent(replica.getBackendId(), k -> Sets.newHashSet()); tabletSet.add(tablet.getId()); } @@ -2524,12 +2723,36 @@ public void updateTableConstraint(Database db, String tableName, Map properties) + throws DdlException { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), tableName); + if (table == null) { + throw new DdlException(String.format("table:%s does not exist", tableName)); + } + OlapTable olapTable = (OlapTable) table; + TableProperty tableProperty = olapTable.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + olapTable.setTableProperty(tableProperty); + } else { + tableProperty.modifyTableProperties(properties); + } + tableProperty.buildConstraint(); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), olapTable.getId(), properties); + GlobalStateMgr.getCurrentState().getEditLog().logModifyConstraint(info); + } + @Override public void cancel(CancelStmt stmt) throws DdlException { cancel(stmt, "user cancelled"); diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java index ad8052e350bff..731cf8c28bbdd 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/SchemaChangeJobV2.java @@ -467,7 +467,7 @@ private void addShadowIndexToCatalog(OlapTable tbl) { Map shadowIndexMap = physicalPartitionIndexMap.row(partitionId); for (MaterializedIndex shadowIndex : shadowIndexMap.values()) { Preconditions.checkState(shadowIndex.getState() == IndexState.SHADOW, shadowIndex.getState()); - partition.createRollupIndex(shadowIndex); + GlobalStateMgr.getCurrentState().getTabletMetastore().addMaterializedIndex(partition, shadowIndex); } } @@ -866,6 +866,9 @@ private void onFinished(OlapTable tbl) { droppedIdx = physicalPartition.getBaseIndex(); } else { droppedIdx = physicalPartition.deleteRollupIndex(originIdxId); + + GlobalStateMgr.getCurrentState().getTabletMetastore().dropMaterializedIndex( + physicalPartition, originIdxId); } Preconditions.checkNotNull(droppedIdx, originIdxId + " vs. " + shadowIdxId); diff --git a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java index ee23d4a73fb50..752ee9e3aa9fa 100644 --- a/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/backup/RestoreJob.java @@ -870,17 +870,20 @@ protected void prepareAndSendSnapshotTasks(Database db) { .getTable(db.getId(), idChain.getTblId()); PhysicalPartition part = tbl.getPhysicalPartition(idChain.getPartId()); MaterializedIndex index = part.getIndex(idChain.getIdxId()); - LocalTablet tablet = (LocalTablet) index.getTablet(idChain.getTabletId()); - Replica replica = tablet.getReplicaById(idChain.getReplicaId()); + + Tablet tablet = GlobalStateMgr.getCurrentState().getTabletMetastore().getTablet(index, idChain.getTabletId()); + Replica replica = GlobalStateMgr.getCurrentState().getTabletMetastore().getReplica( + (LocalTablet) tablet, idChain.getReplicaId()); + long signature = globalStateMgr.getNextId(); SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, jobId, db.getId(), - tbl.getId(), part.getId(), index.getId(), tablet.getId(), + tbl.getId(), part.getId(), index.getId(), idChain.getTabletId(), part.getVisibleVersion(), tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, true /* is restore task*/); batchTask.addTask(task); - unfinishedSignatureToId.put(signature, tablet.getId()); + unfinishedSignatureToId.put(signature, idChain.getTabletId()); bePathsMap.put(replica.getBackendId(), replica.getPathHash()); } } finally { diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/com/starrocks/catalog/CatalogRecycleBin.java index 0c81586fe5959..847876ce82eae 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/CatalogRecycleBin.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/CatalogRecycleBin.java @@ -51,6 +51,8 @@ import com.starrocks.common.io.Text; import com.starrocks.common.io.Writable; import com.starrocks.common.util.FrontendDaemon; +import com.starrocks.common.util.concurrent.lock.LockType; +import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.persist.ImageWriter; import com.starrocks.persist.RecoverInfo; import com.starrocks.persist.gson.IForwardCompatibleObject; @@ -122,7 +124,11 @@ private void removeRecycleMarkers(Long id) { enableEraseLater.remove(id); } - public synchronized void recycleDatabase(Database db, Set tableNames) { + public synchronized void recycleDatabase(Database db, Set tableNames, boolean isForce) { + if (isForce) { + onEraseDatabase(db.getId()); + return; + } Preconditions.checkState(!idToDatabase.containsKey(db.getId())); // db should be empty. all tables are recycled before @@ -138,6 +144,13 @@ public synchronized void recycleDatabase(Database db, Set tableNames) { LOG.info("recycle db[{}-{}]", db.getId(), db.getOriginName()); } + public void onEraseDatabase(long dbId) { + // remove database transaction manager + GlobalStateMgr.getCurrentState().getGlobalTransactionMgr().removeDatabaseTransactionMgr(dbId); + // unbind db to storage volume + GlobalStateMgr.getCurrentState().getStorageVolumeMgr().unbindDbToStorageVolume(dbId); + } + public synchronized Database getDatabase(long dbId) { RecycleDatabaseInfo databaseInfo = idToDatabase.get(dbId); if (databaseInfo != null) { @@ -299,7 +312,7 @@ private synchronized boolean canEraseTable(RecycleTableInfo tableInfo, long curr } // database is force dropped, the table can not be recovered, erase it. - if (GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(tableInfo.getDbId()) == null) { + if (GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(tableInfo.getDbId()) == null) { return true; } return false; @@ -311,13 +324,14 @@ private synchronized boolean canErasePartition(RecyclePartitionInfo partitionInf } // database is force dropped, the partition can not be recovered, erase it. - Database database = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(partitionInfo.getDbId()); + Database database = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getDbIncludeRecycleBin(partitionInfo.getDbId()); if (database == null) { return true; } // table is force dropped, the partition can not be recovered, erase it. - if (GlobalStateMgr.getCurrentState().getLocalMetastore() + if (GlobalStateMgr.getCurrentState().getStarRocksMeta() .getTableIncludeRecycleBin(database, partitionInfo.getTableId()) == null) { return true; } @@ -358,7 +372,7 @@ protected synchronized void eraseDatabase(long currentTimeMs) { dbIter.remove(); removeRecycleMarkers(entry.getKey()); - GlobalStateMgr.getCurrentState().getLocalMetastore().onEraseDatabase(db.getId()); + onEraseDatabase(db.getId()); GlobalStateMgr.getCurrentState().getEditLog().logEraseDb(db.getId()); LOG.info("erase db[{}-{}] finished", db.getId(), db.getOriginName()); currentEraseOpCnt++; @@ -379,7 +393,7 @@ private synchronized void eraseDatabaseWithSameName(String dbName) { iterator.remove(); removeRecycleMarkers(entry.getKey()); - GlobalStateMgr.getCurrentState().getLocalMetastore().onEraseDatabase(db.getId()); + onEraseDatabase(db.getId()); LOG.info("erase database[{}-{}], because db with the same name db is recycled", db.getId(), dbName); } } @@ -389,7 +403,7 @@ public synchronized void replayEraseDatabase(long dbId) { idToDatabase.remove(dbId); idToRecycleTime.remove(dbId); - GlobalStateMgr.getCurrentState().getLocalMetastore().onEraseDatabase(dbId); + onEraseDatabase(dbId); LOG.info("replay erase db[{}] finished", dbId); } @@ -580,7 +594,7 @@ public synchronized void replayErasePartition(long partitionId) { Partition partition = partitionInfo.getPartition(); if (!isCheckpointThread()) { - GlobalStateMgr.getCurrentState().getLocalMetastore().onErasePartition(partition); + GlobalStateMgr.getCurrentState().getStarRocksMeta().onErasePartition(partition); } LOG.info("replay erase partition[{}-{}] finished", partitionId, partition.getName()); @@ -687,18 +701,28 @@ public synchronized boolean recoverTable(Database db, String tableName) { return true; } - public synchronized void replayRecoverTable(Database db, long tableId) { - // make sure to get db write lock - long dbId = db.getId(); - Map idToTableInfoDbLevel = idToTableInfo.row(dbId); - RecycleTableInfo tableInfo = idToTableInfoDbLevel.get(tableId); - Preconditions.checkState(tableInfo.getDbId() == db.getId()); - Table table = tableInfo.getTable(); - db.registerTableUnlocked(table); - nameToTableInfo.row(dbId).remove(table.getName()); - idToTableInfoDbLevel.remove(tableId); - idToRecycleTime.remove(tableInfo.getTable().getId()); - LOG.info("replay recover table[{}-{}] finished", tableId, tableInfo.getTable().getName()); + public synchronized void replayRecoverTable(RecoverInfo recoverInfo) { + long dbId = recoverInfo.getDbId(); + long tableId = recoverInfo.getTableId(); + + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); + Locker locker = new Locker(); + locker.lockDatabase(dbId, LockType.WRITE); + try { + // make sure to get db write lock + Map idToTableInfoDbLevel = idToTableInfo.row(dbId); + RecycleTableInfo tableInfo = idToTableInfoDbLevel.get(tableId); + Preconditions.checkState(tableInfo.getDbId() == db.getId()); + Table table = tableInfo.getTable(); + db.registerTableUnlocked(table); + nameToTableInfo.row(dbId).remove(table.getName()); + idToTableInfoDbLevel.remove(tableId); + idToRecycleTime.remove(tableInfo.getTable().getId()); + LOG.info("replay recover table[{}-{}] finished", tableId, tableInfo.getTable().getName()); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } public synchronized void recoverPartition(long dbId, OlapTable table, String partitionName) throws DdlException { @@ -783,6 +807,7 @@ public void addTabletToInvertedIndex() { // no need to handle idToDatabase. Database is already empty before being put here TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + // idToTable for (RecycleTableInfo tableInfo : idToTableInfo.values()) { Table table = tableInfo.getTable(); @@ -808,6 +833,7 @@ public void addTabletToInvertedIndex() { int schemaHash = olapTable.getSchemaHashByIndexId(indexId); TabletMeta tabletMeta = new TabletMeta(dbId, tableId, physicalPartitionId, indexId, schemaHash, medium, table.isCloudNativeTable()); + for (Tablet tablet : index.getTablets()) { long tabletId = tablet.getId(); invertedIndex.addTablet(tabletId, tabletMeta); diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/ColocateTableIndex.java b/fe/fe-core/src/main/java/com/starrocks/catalog/ColocateTableIndex.java index c587f98c7e9cb..773a2e230772e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/ColocateTableIndex.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/ColocateTableIndex.java @@ -1008,12 +1008,12 @@ protected void cleanupInvalidDbOrTable(GlobalStateMgr globalStateMgr) { for (Map.Entry entry : table2Group.entrySet()) { long dbId = entry.getValue().dbId; long tableId = entry.getKey(); - Database database = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database database = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (database == null) { LOG.warn("cannot find db {}, will remove invalid table {} from group {}", dbId, tableId, entry.getValue()); } else { - Table table = globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(database, tableId); + Table table = globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(database, tableId); if (table != null) { // this is a valid table/database, do nothing continue; @@ -1064,8 +1064,8 @@ private void constructLakeGroups(GlobalStateMgr globalStateMgr) { long dbId = entry.getValue().dbId; long tableId = entry.getKey(); // database and table should be valid if reach here - Database database = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); - Table table = globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(database, tableId); + Database database = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); + Table table = globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(database, tableId); if (table.isCloudNativeTable()) { lakeGroups.add(entry.getValue()); } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/Database.java b/fe/fe-core/src/main/java/com/starrocks/catalog/Database.java index 275ed6e0cb29d..d74a712d37095 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/Database.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/Database.java @@ -334,7 +334,7 @@ public void dropTable(String tableName, boolean isSetIfExists, boolean isForce) } unprotectDropTable(table.getId(), isForce, false); DropInfo info = new DropInfo(id, table.getId(), -1L, isForce); - GlobalStateMgr.getCurrentState().getEditLog().logDropTable(info); + GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(info); } finally { locker.unLockDatabase(id, LockType.WRITE); } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/LocalTablet.java b/fe/fe-core/src/main/java/com/starrocks/catalog/LocalTablet.java index d3c7f0ae6ed9c..e5c9dd89137e4 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/LocalTablet.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/LocalTablet.java @@ -53,9 +53,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -405,47 +402,6 @@ public void clearReplica() { } } - @Override - public void write(DataOutput out) throws IOException { - super.write(out); - - out.writeLong(id); - int replicaCount = replicas.size(); - out.writeInt(replicaCount); - for (int i = 0; i < replicaCount; ++i) { - replicas.get(i).write(out); - } - - out.writeLong(checkedVersion); - out.writeLong(0); // write a version_hash for compatibility - out.writeBoolean(isConsistent); - } - - @Override - public void readFields(DataInput in) throws IOException { - super.readFields(in); - - id = in.readLong(); - int replicaCount = in.readInt(); - for (int i = 0; i < replicaCount; ++i) { - Replica replica = Replica.read(in); - if (deleteRedundantReplica(replica.getBackendId(), replica.getVersion())) { - // do not need to update immutableReplicas, because it is a view of replicas - replicas.add(replica); - } - } - - checkedVersion = in.readLong(); - in.readLong(); // read a version_hash for compatibility - isConsistent = in.readBoolean(); - } - - public static LocalTablet read(DataInput in) throws IOException { - LocalTablet tablet = new LocalTablet(); - tablet.readFields(in); - return tablet; - } - @Override public void gsonPostProcess() { // we need to update immutableReplicas, because replicas after deserialization from a json string diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/MaterializedIndex.java b/fe/fe-core/src/main/java/com/starrocks/catalog/MaterializedIndex.java index 5a9848463c2df..d75091e34730b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/MaterializedIndex.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/MaterializedIndex.java @@ -48,9 +48,9 @@ import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import javax.annotation.Nullable; public class MaterializedIndex extends MetaObject implements Writable, GsonPostProcessable { @@ -136,7 +136,7 @@ public MaterializedIndex(long id, @Nullable IndexState state) { public MaterializedIndex(long id, @Nullable IndexState state, long visibleTxnId) { this.id = id; this.state = state == null ? IndexState.NORMAL : state; - this.idToTablets = new HashMap<>(); + this.idToTablets = new TreeMap<>(); this.tablets = new ArrayList<>(); this.rowCount = 0; this.visibleTxnId = (this.state == IndexState.SHADOW) ? visibleTxnId : 0; @@ -227,7 +227,7 @@ public void setRowCount(long rowCount) { public long getDataSize() { long dataSize = 0; - for (Tablet tablet : getTablets()) { + for (Tablet tablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(this)) { dataSize += tablet.getDataSize(false); } return dataSize; @@ -235,7 +235,7 @@ public long getDataSize() { public long getTabletMaxDataSize() { long maxDataSize = 0; - for (Tablet tablet : getTablets()) { + for (Tablet tablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(this)) { maxDataSize = Math.max(maxDataSize, tablet.getDataSize(true)); } return maxDataSize; @@ -252,7 +252,7 @@ public long getReplicaCount() { } else { Preconditions.checkState(t instanceof LocalTablet); long replicaCount = 0; - for (Tablet tablet : getTablets()) { + for (Tablet tablet : GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(this)) { LocalTablet localTablet = (LocalTablet) tablet; replicaCount += localTablet.getImmutableReplicas().size(); } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/com/starrocks/catalog/MetadataViewer.java index d72855d1d8cea..8792c523e73d9 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/MetadataViewer.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/MetadataViewer.java @@ -44,6 +44,7 @@ import com.starrocks.common.FeConstants; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.meta.TabletMetastore; import com.starrocks.qe.ConnectContext; import com.starrocks.server.GlobalStateMgr; import com.starrocks.server.RunMode; @@ -72,6 +73,7 @@ private static List> getTabletStatus(String dbName, String tblName, GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); SystemInfoService infoService = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); Database db = globalStateMgr.getLocalMetastore().getDb(dbName); if (db == null) { @@ -109,7 +111,7 @@ private static List> getTabletStatus(String dbName, String tblName, for (MaterializedIndex index : physicalPartition.getMaterializedIndices(IndexExtState.VISIBLE)) { int schemaHash = olapTable.getSchemaHashByIndexId(index.getId()); - for (Tablet tablet : index.getTablets()) { + for (Tablet tablet : tabletMetastore.getAllTablets(index)) { long tabletId = tablet.getId(); int count = replicationNum; for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java index 3adb93b7577b1..41e9afe611693 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java @@ -90,11 +90,12 @@ import com.starrocks.lake.DataCacheInfo; import com.starrocks.lake.StarOSAgent; import com.starrocks.lake.StorageInfo; +import com.starrocks.meta.StarRocksMeta; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ColocatePersistInfo; import com.starrocks.qe.ConnectContext; import com.starrocks.qe.OriginStatement; import com.starrocks.server.GlobalStateMgr; -import com.starrocks.server.LocalMetastore; import com.starrocks.server.RunMode; import com.starrocks.server.TemporaryTableMgr; import com.starrocks.sql.analyzer.AnalyzeState; @@ -195,7 +196,7 @@ public enum OlapTableState { * or ROLLUP). * The query plan which is generate during this state is invalid because the meta * during the creation of the logical plan and the physical plan might be inconsistent. - */ + */ UPDATING_META } @@ -944,11 +945,11 @@ public Status createTabletsForRestore(int tabletNum, MaterializedIndex index, Gl if (isColocate) { try { isColocate = GlobalStateMgr.getCurrentState().getColocateTableIndex() - .addTableToGroup(db, this, this.colocateGroup, false); + .addTableToGroup(db, this, this.colocateGroup, false); } catch (Exception e) { return new Status(ErrCode.COMMON_ERROR, - "check colocate restore failed, errmsg: " + e.getMessage() + - ", you can disable colocate restore by turn off Config.enable_colocate_restore"); + "check colocate restore failed, errmsg: " + e.getMessage() + + ", you can disable colocate restore by turn off Config.enable_colocate_restore"); } } @@ -966,14 +967,13 @@ public Status createTabletsForRestore(int tabletNum, MaterializedIndex index, Gl long newTabletId = globalStateMgr.getNextId(); LocalTablet newTablet = new LocalTablet(newTabletId); index.addTablet(newTablet, null /* tablet meta */, false/* update inverted index */); - // replicas List beIds; if (chooseBackendsArbitrary) { // This is the first colocate table in the group, or just a normal table, // randomly choose backends beIds = GlobalStateMgr.getCurrentState().getNodeMgr() - .getClusterInfo().getNodeSelector().seqChooseBackendIds(replicationNum, true, true, getLocation()); + .getClusterInfo().getNodeSelector().seqChooseBackendIds(replicationNum, true, true, getLocation()); backendsPerBucketSeq.add(beIds); } else { // get backends from existing backend sequence @@ -992,7 +992,7 @@ public Status createTabletsForRestore(int tabletNum, MaterializedIndex index, Gl newTablet.addReplica(replica, false/* update inverted index */); } Preconditions.checkState(beIds.size() == replicationNum, - beIds.size() + " vs. " + replicationNum); + beIds.size() + " vs. " + replicationNum); } // first colocate table in CG @@ -1236,8 +1236,8 @@ public Map> getRangePartitionMap() { /** * @return : table's partition name to list partition names. * eg: - * partition columns : (a, b, c) - * values : [[1, 2, 3], [4, 5, 6]] + * partition columns : (a, b, c) + * values : [[1, 2, 3], [4, 5, 6]] */ public Map getListPartitionItems() { Preconditions.checkState(partitionInfo instanceof ListPartitionInfo); @@ -2248,6 +2248,7 @@ public long checkAndGetUnhealthyTablet(SystemInfoService infoService, TabletSche // arbitrarily choose a partition, and get the buckets backends sequence from // base index. public List> getArbitraryTabletBucketsSeq() throws DdlException { + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); List> backendsPerBucketSeq = Lists.newArrayList(); Optional optionalPartition = idToPartition.values().stream().findFirst(); if (optionalPartition.isPresent()) { @@ -2255,7 +2256,7 @@ public List> getArbitraryTabletBucketsSeq() throws DdlException { short replicationNum = partitionInfo.getReplicationNum(partition.getId()); MaterializedIndex baseIdx = partition.getBaseIndex(); for (Long tabletId : baseIdx.getTabletIdsInOrder()) { - LocalTablet tablet = (LocalTablet) baseIdx.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) tabletMetastore.getTablet(baseIdx, tabletId); List replicaBackendIds = tablet.getNormalReplicaBackendIds(); if (replicaBackendIds.size() < replicationNum) { // this should not happen, but in case, throw an exception to terminate this @@ -3167,7 +3168,7 @@ public void onDrop(Database db, boolean force, boolean replay) { // in recycle bin, // which make things easier. dropAllTempPartitions(); - LocalMetastore.inactiveRelatedMaterializedView(db, this, + StarRocksMeta.inactiveRelatedMaterializedView(db, this, MaterializedViewExceptions.inactiveReasonForBaseTableNotExists(getName())); if (!replay && hasAutoIncrementColumn()) { sendDropAutoIncrementMapTask(); diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/PhysicalPartition.java b/fe/fe-core/src/main/java/com/starrocks/catalog/PhysicalPartition.java index 8e524d9758c62..747508fa9574a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/PhysicalPartition.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/PhysicalPartition.java @@ -14,92 +14,476 @@ package com.starrocks.catalog; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.annotations.SerializedName; import com.starrocks.catalog.MaterializedIndex.IndexExtState; +import com.starrocks.catalog.MaterializedIndex.IndexState; +import com.starrocks.common.FeConstants; +import com.starrocks.persist.gson.GsonPostProcessable; +import com.starrocks.server.GlobalStateMgr; import com.starrocks.transaction.TransactionType; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicBoolean; -/* - * PhysicalPartition is the interface that describes the physical storage of a partition. - * It includes version information and one or more MaterializedIndexes. - * Each MaterializedIndex contains multiple tablets. +/** + * Physical Partition implementation */ -public interface PhysicalPartition { - - // partition id which contains this physical partition - public long getParentId(); - public void setParentId(long parentId); - - // physical partition id - public long getId(); - public String getName(); - public void setName(String name); - public void setIdForRestore(long id); - public long getBeforeRestoreId(); - - public long getShardGroupId(); - - public void setImmutable(boolean isImmutable); - public boolean isImmutable(); - - // version interface - - public void updateVersionForRestore(long visibleVersion); - - public void updateVisibleVersion(long visibleVersion); - - public void updateVisibleVersion(long visibleVersion, long visibleVersionTime); - public void updateVisibleVersion(long visibleVersion, long visibleVersionTime, long visibleTxnId); - public long getVisibleVersion(); - public long getVisibleVersionTime(); - public void setVisibleVersion(long visibleVersion, long visibleVersionTime); - public long getNextVersion(); - public void setNextVersion(long nextVersion); - public long getCommittedVersion(); - public long getDataVersion(); - public void setDataVersion(long dataVersion); - public long getNextDataVersion(); - public void setNextDataVersion(long nextDataVersion); - public long getCommittedDataVersion(); - public long getVersionEpoch(); - public void setVersionEpoch(long versionEpoch); - public long nextVersionEpoch(); - public TransactionType getVersionTxnType(); - public void setVersionTxnType(TransactionType versionTxnType); - public long getVisibleTxnId(); - - // materialized index interface - - public void createRollupIndex(MaterializedIndex mIndex); - public MaterializedIndex deleteRollupIndex(long indexId); - public void setBaseIndex(MaterializedIndex baseIndex); - public MaterializedIndex getBaseIndex(); - public MaterializedIndex getIndex(long indexId); - public List getMaterializedIndices(IndexExtState extState); +public class PhysicalPartition extends MetaObject implements GsonPostProcessable { + private static final Logger LOG = LogManager.getLogger(PhysicalPartition.class); + + public static final long PARTITION_INIT_VERSION = 1L; + + @SerializedName(value = "id") + private long id; + + @SerializedName(value = "name") + private String name; + + private long beforeRestoreId; + + @SerializedName(value = "parentId") + private long parentId; + + @SerializedName(value = "shardGroupId") + private long shardGroupId; + + /* Physical Partition Member */ + @SerializedName(value = "isImmutable") + private AtomicBoolean isImmutable = new AtomicBoolean(false); + + @SerializedName(value = "baseIndex") + private MaterializedIndex baseIndex; + /** + * Visible rollup indexes are indexes which are visible to user. + * User can do query on them, show them in related 'show' stmt. + */ + @SerializedName(value = "idToVisibleRollupIndex") + private Map idToVisibleRollupIndex = Maps.newHashMap(); + /** + * Shadow indexes are indexes which are not visible to user. + * Query will not run on these shadow indexes, and user can not see them neither. + * But load process will load data into these shadow indexes. + */ + @SerializedName(value = "idToShadowIndex") + private Map idToShadowIndex = Maps.newHashMap(); + + /** + * committed version(hash): after txn is committed, set committed version(hash) + * visible version(hash): after txn is published, set visible version + * next version(hash): next version is set after finished committing, it should equal to committed version + 1 + */ + + // not have committedVersion because committedVersion = nextVersion - 1 + @SerializedName(value = "visibleVersion") + private long visibleVersion; + @SerializedName(value = "visibleVersionTime") + private long visibleVersionTime; + @SerializedName(value = "nextVersion") + private long nextVersion; + + @SerializedName(value = "dataVersion") + private long dataVersion; + @SerializedName(value = "nextDataVersion") + private long nextDataVersion; + + @SerializedName(value = "versionEpoch") + private long versionEpoch; + @SerializedName(value = "versionTxnType") + private TransactionType versionTxnType; + /** + * ID of the transaction that has committed current visible version. + * Just for tracing the txn log, no need to persist. + */ + private long visibleTxnId = -1; + + private volatile long lastVacuumTime = 0; + + private volatile long minRetainVersion = 0; + + public PhysicalPartition(long id, String name, long parentId, long sharedGroupId, MaterializedIndex baseIndex) { + this.id = id; + this.name = name; + this.parentId = parentId; + this.baseIndex = baseIndex; + this.visibleVersion = PARTITION_INIT_VERSION; + this.visibleVersionTime = System.currentTimeMillis(); + this.nextVersion = this.visibleVersion + 1; + this.dataVersion = this.visibleVersion; + this.nextDataVersion = this.nextVersion; + this.versionEpoch = this.nextVersionEpoch(); + this.versionTxnType = TransactionType.TXN_NORMAL; + this.shardGroupId = sharedGroupId; + } + + public long getId() { + return this.id; + } + + public String getName() { + return this.name; + } + + public void setName(String name) { + this.name = name; + } + + public void setIdForRestore(long id) { + this.beforeRestoreId = this.id; + this.id = id; + } + + public long getBeforeRestoreId() { + return this.beforeRestoreId; + } + + public long getParentId() { + return this.parentId; + } + + public void setParentId(long parentId) { + this.parentId = parentId; + } + + public long getShardGroupId() { + return this.shardGroupId; + } + + public void setImmutable(boolean isImmutable) { + this.isImmutable.set(isImmutable); + } + + public boolean isImmutable() { + return this.isImmutable.get(); + } + + public long getLastVacuumTime() { + return lastVacuumTime; + } + + public void setLastVacuumTime(long lastVacuumTime) { + this.lastVacuumTime = lastVacuumTime; + } + + public long getMinRetainVersion() { + return minRetainVersion; + } + + public void setMinRetainVersion(long minRetainVersion) { + this.minRetainVersion = minRetainVersion; + } + + /* + * If a partition is overwritten by a restore job, we need to reset all version info to + * the restored partition version info) + */ + + public void updateVersionForRestore(long visibleVersion) { + this.setVisibleVersion(visibleVersion, System.currentTimeMillis()); + this.nextVersion = this.visibleVersion + 1; + LOG.info("update partition {} version for restore: visible: {}, next: {}", + id, visibleVersion, nextVersion); + } + + public void updateVisibleVersion(long visibleVersion) { + updateVisibleVersion(visibleVersion, System.currentTimeMillis()); + } + + public void updateVisibleVersion(long visibleVersion, long visibleVersionTime) { + this.setVisibleVersion(visibleVersion, visibleVersionTime); + } + + public void updateVisibleVersion(long visibleVersion, long visibleVersionTime, long visibleTxnId) { + setVisibleVersion(visibleVersion, visibleVersionTime); + this.visibleTxnId = visibleTxnId; + } + + public long getVisibleTxnId() { + return visibleTxnId; + } + + public long getVisibleVersion() { + return visibleVersion; + } + + public long getVisibleVersionTime() { + return visibleVersionTime; + } + + public void setVisibleVersion(long visibleVersion, long visibleVersionTime) { + this.visibleVersion = visibleVersion; + this.visibleVersionTime = visibleVersionTime; + } + + public void createRollupIndex(MaterializedIndex mIndex) { + if (mIndex.getState().isVisible()) { + this.idToVisibleRollupIndex.put(mIndex.getId(), mIndex); + } else { + this.idToShadowIndex.put(mIndex.getId(), mIndex); + } + } + + public MaterializedIndex deleteRollupIndex(long indexId) { + if (this.idToVisibleRollupIndex.containsKey(indexId)) { + return idToVisibleRollupIndex.remove(indexId); + } else { + return idToShadowIndex.remove(indexId); + } + } + + public void setBaseIndex(MaterializedIndex baseIndex) { + this.baseIndex = baseIndex; + } + + public MaterializedIndex getBaseIndex() { + return baseIndex; + } + + public long getNextVersion() { + return nextVersion; + } + + public void setNextVersion(long nextVersion) { + this.nextVersion = nextVersion; + } + + public long getCommittedVersion() { + return this.nextVersion - 1; + } + + public long getDataVersion() { + return dataVersion; + } + + public void setDataVersion(long dataVersion) { + this.dataVersion = dataVersion; + } + + public long getNextDataVersion() { + return nextDataVersion; + } + + public void setNextDataVersion(long nextDataVersion) { + this.nextDataVersion = nextDataVersion; + } + + public long getCommittedDataVersion() { + return this.nextDataVersion - 1; + } + + public long getVersionEpoch() { + return versionEpoch; + } + + public void setVersionEpoch(long versionEpoch) { + this.versionEpoch = versionEpoch; + } + + public long nextVersionEpoch() { + return GlobalStateMgr.getCurrentState().getGtidGenerator().nextGtid(); + } + + public TransactionType getVersionTxnType() { + return versionTxnType; + } + + public void setVersionTxnType(TransactionType versionTxnType) { + this.versionTxnType = versionTxnType; + } + + public MaterializedIndex getIndex(long indexId) { + if (baseIndex.getId() == indexId) { + return baseIndex; + } + if (idToVisibleRollupIndex.containsKey(indexId)) { + return idToVisibleRollupIndex.get(indexId); + } else { + return idToShadowIndex.get(indexId); + } + } + + public List getMaterializedIndices(IndexExtState extState) { + int expectedSize = 1 + idToVisibleRollupIndex.size() + idToShadowIndex.size(); + List indices = Lists.newArrayListWithExpectedSize(expectedSize); + switch (extState) { + case ALL: + indices.add(baseIndex); + indices.addAll(idToVisibleRollupIndex.values()); + indices.addAll(idToShadowIndex.values()); + break; + case VISIBLE: + indices.add(baseIndex); + indices.addAll(idToVisibleRollupIndex.values()); + break; + case SHADOW: + indices.addAll(idToShadowIndex.values()); + default: + break; + } + return indices; + } + + public long getTabletMaxDataSize() { + long maxDataSize = 0; + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { + maxDataSize = Math.max(maxDataSize, mIndex.getTabletMaxDataSize()); + } + return maxDataSize; + } + + public long storageDataSize() { + long dataSize = 0; + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { + dataSize += mIndex.getDataSize(); + } + return dataSize; + } + + public long storageRowCount() { + long rowCount = 0; + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { + rowCount += mIndex.getRowCount(); + } + return rowCount; + } + + public long storageReplicaCount() { + long replicaCount = 0; + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { + replicaCount += mIndex.getReplicaCount(); + } + return replicaCount; + } + + public boolean hasMaterializedView() { + return !idToVisibleRollupIndex.isEmpty(); + } + + public boolean hasStorageData() { + // The fe unit test need to check the selected index id without any data. + // So if set FeConstants.runningUnitTest, we can ensure that the number of partitions is not empty, + // And the test case can continue to execute the logic of 'select best roll up' + return ((visibleVersion != PARTITION_INIT_VERSION) + || FeConstants.runningUnitTest); + } + + public boolean isFirstLoad() { + return visibleVersion == PARTITION_INIT_VERSION + 1; + } + /* * Change the index' state from SHADOW to NORMAL + * Also move it to idToVisibleRollupIndex if it is not the base index. */ - public boolean visualiseShadowIndex(long shadowIndexId, boolean isBaseIndex); - - // statistic interface - - // max data size of one tablet in this physical partition - public long getTabletMaxDataSize(); - // partition data size reported by be, but may be not accurate - public long storageDataSize(); - // partition row count reported by be, but may be not accurate - public long storageRowCount(); - // partition replica count, it's accurate - public long storageReplicaCount(); - // has data judge by fe version, it's accurate - public boolean hasStorageData(); - public boolean hasMaterializedView(); - public boolean isFirstLoad(); - - // for lake partition - public long getMinRetainVersion(); - public void setMinRetainVersion(long minRetainVersion); - public long getLastVacuumTime(); - public void setLastVacuumTime(long lastVacuumTime); + public boolean visualiseShadowIndex(long shadowIndexId, boolean isBaseIndex) { + MaterializedIndex shadowIdx = idToShadowIndex.remove(shadowIndexId); + if (shadowIdx == null) { + return false; + } + Preconditions.checkState(!idToVisibleRollupIndex.containsKey(shadowIndexId), shadowIndexId); + shadowIdx.setState(IndexState.NORMAL); + if (isBaseIndex) { + baseIndex = shadowIdx; + } else { + idToVisibleRollupIndex.put(shadowIndexId, shadowIdx); + } + LOG.info("visualise the shadow index: {}", shadowIndexId); + return true; + } + + public String generatePhysicalPartitionName(long physicalParitionId) { + return this.name + '_' + physicalParitionId; + } + + public int hashCode() { + return Objects.hashCode(visibleVersion, baseIndex); + } + + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof PhysicalPartition)) { + return false; + } + + PhysicalPartition partition = (PhysicalPartition) obj; + if (idToVisibleRollupIndex != partition.idToVisibleRollupIndex) { + if (idToVisibleRollupIndex.size() != partition.idToVisibleRollupIndex.size()) { + return false; + } + for (Entry entry : idToVisibleRollupIndex.entrySet()) { + long key = entry.getKey(); + if (!partition.idToVisibleRollupIndex.containsKey(key)) { + return false; + } + if (!entry.getValue().equals(partition.idToVisibleRollupIndex.get(key))) { + return false; + } + } + } + + return (visibleVersion == partition.visibleVersion) + && (baseIndex.equals(partition.baseIndex)); + } + + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("partitionId: ").append(id).append("; "); + buffer.append("partitionName: ").append(name).append("; "); + buffer.append("parentPartitionId: ").append(parentId).append("; "); + buffer.append("shardGroupId: ").append(shardGroupId).append("; "); + buffer.append("isImmutable: ").append(isImmutable()).append("; "); + + buffer.append("baseIndex: ").append(baseIndex.toString()).append("; "); + + int rollupCount = (idToVisibleRollupIndex != null) ? idToVisibleRollupIndex.size() : 0; + buffer.append("rollupCount: ").append(rollupCount).append("; "); + + if (idToVisibleRollupIndex != null) { + for (Map.Entry entry : idToVisibleRollupIndex.entrySet()) { + buffer.append("rollupIndex: ").append(entry.getValue().toString()).append("; "); + } + } + + buffer.append("visibleVersion: ").append(visibleVersion).append("; "); + buffer.append("visibleVersionTime: ").append(visibleVersionTime).append("; "); + buffer.append("committedVersion: ").append(getCommittedVersion()).append("; "); + + buffer.append("dataVersion: ").append(dataVersion).append("; "); + buffer.append("committedDataVersion: ").append(getCommittedDataVersion()).append("; "); + + buffer.append("versionEpoch: ").append(versionEpoch).append("; "); + buffer.append("versionTxnType: ").append(versionTxnType).append("; "); + + buffer.append("storageDataSize: ").append(storageDataSize()).append("; "); + buffer.append("storageRowCount: ").append(storageRowCount()).append("; "); + buffer.append("storageReplicaCount: ").append(storageReplicaCount()).append("; "); + + return buffer.toString(); + } + + public void gsonPostProcess() throws IOException { + if (dataVersion == 0) { + dataVersion = visibleVersion; + } + if (nextDataVersion == 0) { + nextDataVersion = nextVersion; + } + if (versionEpoch == 0) { + versionEpoch = nextVersionEpoch(); + } + if (versionTxnType == null) { + versionTxnType = TransactionType.TXN_NORMAL; + } + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/RecyclePartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/catalog/RecyclePartitionInfo.java index ae8be23fbe2e6..140b68c5ba255 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/RecyclePartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/RecyclePartitionInfo.java @@ -91,7 +91,7 @@ public void setRecoverable(boolean recoverable) { } public boolean delete() { - GlobalStateMgr.getCurrentState().getLocalMetastore().onErasePartition(partition); + GlobalStateMgr.getCurrentState().getStarRocksMeta().onErasePartition(partition); return true; } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletInvertedIndex.java index 0a8667e0931b6..2f8584067b7c5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletInvertedIndex.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletInvertedIndex.java @@ -57,6 +57,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; +import static com.starrocks.server.GlobalStateMgr.isCheckpointThread; + /* * this class stores an inverted index * key is tablet id. value is the related ids of this tablet @@ -432,11 +434,52 @@ public void clear() { } } + public void recreateTabletInvertIndex() { + if (isCheckpointThread()) { + return; + } + + // create inverted index + TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + for (Database db : GlobalStateMgr.getCurrentState().getLocalMetastore().getFullNameToDb().values()) { + long dbId = db.getId(); + for (com.starrocks.catalog.Table table : db.getTables()) { + if (!table.isNativeTableOrMaterializedView()) { + continue; + } + + OlapTable olapTable = (OlapTable) table; + long tableId = olapTable.getId(); + for (PhysicalPartition partition : olapTable.getAllPhysicalPartitions()) { + long physicalPartitionId = partition.getId(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( + partition.getParentId()).getStorageMedium(); + for (MaterializedIndex index : partition + .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + long indexId = index.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partition.getParentId(), physicalPartitionId, + indexId, schemaHash, medium, table.isCloudNativeTableOrMaterializedView()); + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.addTablet(tabletId, tabletMeta); + if (table.isOlapTableOrMaterializedView()) { + for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { + invertedIndex.addReplica(tabletId, replica); + } + } + } + } // end for indices + } // end for partitions + } // end for tables + } // end for dbs + } + @Override public Map estimateCount() { return ImmutableMap.of("TabletMeta", (long) tabletMetaMap.size(), - "TabletCount", getTabletCount(), - "ReplicateCount", getReplicaCount()); + "TabletCount", getTabletCount(), + "ReplicateCount", getReplicaCount()); } @Override diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletMeta.java b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletMeta.java index 9743b026d0cf7..b12df5527ea8e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletMeta.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletMeta.java @@ -44,6 +44,7 @@ public class TabletMeta { private final long partitionId; private final long physicalPartitionId; private final long indexId; + private final long tabletId = 0; private final int oldSchemaHash; private final int newSchemaHash; diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletStatMgr.java index ddaa5de0aef41..7763bc5aa0c90 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/TabletStatMgr.java @@ -119,8 +119,12 @@ protected void runAfterCatalogReady() { Map indexRowCountMap = Maps.newHashMap(); try { OlapTable olapTable = (OlapTable) table; - for (Partition partition : olapTable.getAllPartitions()) { - for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + List partitionList = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getAllPartitions(db, olapTable); + for (Partition partition : partitionList) { + List physicalPartitionList = GlobalStateMgr.getCurrentState().getTabletMetastore() + .getAllPhysicalPartition(partition); + for (PhysicalPartition physicalPartition : physicalPartitionList) { long version = physicalPartition.getVisibleVersion(); for (MaterializedIndex index : physicalPartition.getMaterializedIndices( IndexExtState.VISIBLE)) { diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/ColocateTableBalancer.java b/fe/fe-core/src/main/java/com/starrocks/clone/ColocateTableBalancer.java index 6c2d3c00c0e21..5b5a4f2f2b707 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/ColocateTableBalancer.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/ColocateTableBalancer.java @@ -328,7 +328,7 @@ private boolean relocateAndBalancePerGroup() { Set toIgnoreGroupIds = new HashSet<>(); boolean isAnyGroupChanged = false; for (GroupId groupId : groupIds) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(groupId.dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(groupId.dbId); if (db == null) { continue; } @@ -726,7 +726,7 @@ private ColocateMatchResult doMatchOneGroup(GroupId groupId, long lockTotalTime = 0; long waitTotalTimeMs = 0; List tableIds = colocateIndex.getAllTableIds(groupId); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(groupId.dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(groupId.dbId); if (db == null) { return new ColocateMatchResult(lockTotalTime, Status.UNKNOWN); } @@ -746,7 +746,7 @@ private ColocateMatchResult doMatchOneGroup(GroupId groupId, try { TABLE: for (Long tableId : tableIds) { - OlapTable olapTable = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable olapTable = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tableId); if (olapTable == null || !colocateIndex.isColocateTable(olapTable.getId())) { continue; } @@ -755,7 +755,7 @@ private ColocateMatchResult doMatchOneGroup(GroupId groupId, continue; } - for (Partition partition : globalStateMgr.getLocalMetastore().getPartitionsIncludeRecycleBin(olapTable)) { + for (Partition partition : globalStateMgr.getStarRocksMeta().getPartitionsIncludeRecycleBin(olapTable)) { partitionChecked++; boolean isPartitionUrgent = @@ -771,20 +771,20 @@ private ColocateMatchResult doMatchOneGroup(GroupId groupId, locker.unLockDatabase(db.getId(), LockType.READ); locker.lockDatabase(db.getId(), LockType.READ); lockStart = System.nanoTime(); - if (globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(groupId.dbId) == null) { + if (globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(groupId.dbId) == null) { return new ColocateMatchResult(lockTotalTime, Status.UNKNOWN); } - if (globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, olapTable.getId()) == null) { + if (globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, olapTable.getId()) == null) { continue TABLE; } - if (globalStateMgr.getLocalMetastore().getPartitionIncludeRecycleBin(olapTable, partition.getId()) == + if (globalStateMgr.getStarRocksMeta().getPartitionIncludeRecycleBin(olapTable, partition.getId()) == null) { continue; } } short replicationNum = - globalStateMgr.getLocalMetastore().getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), - partition.getId()); + globalStateMgr.getStarRocksMeta() + .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), partition.getId()); if (replicationNum == (short) -1) { continue; } diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java b/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java index 66e90822f6f9b..7f9475e6eacb7 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/DiskAndTabletLoadReBalancer.java @@ -37,6 +37,7 @@ import com.starrocks.clone.BackendLoadStatistic.Classification; import com.starrocks.common.Config; import com.starrocks.common.Pair; +import com.starrocks.common.util.concurrent.lock.AutoCloseableLock; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.server.GlobalStateMgr; @@ -706,7 +707,7 @@ private List balanceBackendDisk(ClusterLoadStatistic clusterStat private OlapTable getOlapTableById(long dbId, long tblId) { GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { return null; } @@ -714,7 +715,7 @@ private OlapTable getOlapTableById(long dbId, long tblId) { Locker locker = new Locker(); try { locker.lockDatabase(db.getId(), LockType.READ); - return (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tblId); + return (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tblId); } finally { locker.unLockDatabase(db.getId(), LockType.READ); } @@ -1005,7 +1006,7 @@ private Map, Double> getPartitionAvgReplicaSize(long beId, private int getPartitionTabletNumOnBePath(long dbId, long tableId, long partitionId, long indexId, long beId, long pathHash) { GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { return 0; } @@ -1013,12 +1014,12 @@ private int getPartitionTabletNumOnBePath(long dbId, long tableId, long partitio Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.READ); try { - OlapTable table = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable table = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tableId); if (table == null) { return 0; } - Partition partition = globalStateMgr.getLocalMetastore().getPartitionIncludeRecycleBin(table, partitionId); + Partition partition = globalStateMgr.getStarRocksMeta().getPartitionIncludeRecycleBin(table, partitionId); if (partition == null) { return 0; } @@ -1341,7 +1342,7 @@ private boolean isDestBackendLocationMismatch(OlapTable olapTable, long partitionId, long tabletId) { short replicationFactor = - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), partitionId); if (replicationFactor == (short) -1) { return true; @@ -1451,14 +1452,14 @@ private List>> getPartitionTablets(Long dbId, Long tableId, List>> result = Lists.newArrayList(); GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { return result; } Locker locker = new Locker(); try { locker.lockDatabase(db.getId(), LockType.READ); - OlapTable table = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable table = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tableId); if (table == null) { return result; } @@ -1467,7 +1468,7 @@ private List>> getPartitionTablets(Long dbId, Long tableId, return result; } - Partition partition = globalStateMgr.getLocalMetastore().getPartitionIncludeRecycleBin(table, partitionId); + Partition partition = globalStateMgr.getStarRocksMeta().getPartitionIncludeRecycleBin(table, partitionId); if (partition == null) { return result; } @@ -1538,15 +1539,9 @@ private List>> getPartitionTablets(Long dbId, Long tableId, private boolean isTabletUnhealthy(long dbId, OlapTable olapTable, Long tabletId, TabletMeta tabletMeta, List aliveBeIds) { GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); - if (db == null) { - return false; - } - - Locker locker = new Locker(); - try { - locker.lockDatabase(db.getId(), LockType.READ); - Partition partition = globalStateMgr.getLocalMetastore() + try (AutoCloseableLock ignore = + new AutoCloseableLock(new Locker(), dbId, Lists.newArrayList(olapTable.getId()), LockType.READ)) { + Partition partition = globalStateMgr.getStarRocksMeta() .getPartitionIncludeRecycleBin(olapTable, tabletMeta.getPartitionId()); if (partition == null) { return true; @@ -1557,12 +1552,12 @@ private boolean isTabletUnhealthy(long dbId, OlapTable olapTable, Long tabletId, return true; } - LocalTablet tablet = (LocalTablet) index.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) GlobalStateMgr.getCurrentState().getTabletMetastore().getTablet(index, tabletId); if (tablet == null) { return true; } - short replicaNum = globalStateMgr.getLocalMetastore() + short replicaNum = globalStateMgr.getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), partition.getId()); if (replicaNum == (short) -1) { return true; @@ -1579,8 +1574,6 @@ private boolean isTabletUnhealthy(long dbId, OlapTable olapTable, Long tabletId, return statusPair.first != LocalTablet.TabletHealthStatus.LOCATION_MISMATCH && statusPair.first != LocalTablet.TabletHealthStatus.HEALTHY; - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); } } @@ -1607,10 +1600,10 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me long start = System.nanoTime(); long lockTotalTime = 0; long lockStart; - List dbIds = globalStateMgr.getLocalMetastore().getDbIdsIncludeRecycleBin(); + List dbIds = globalStateMgr.getStarRocksMeta().getDbIdsIncludeRecycleBin(); DATABASE: for (Long dbId : dbIds) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -1627,7 +1620,7 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me lockStart = System.nanoTime(); try { TABLE: - for (Table table : globalStateMgr.getLocalMetastore().getTablesIncludeRecycleBin(db)) { + for (Table table : globalStateMgr.getStarRocksMeta().getTablesIncludeRecycleBin(db)) { // check table is olap table or colocate table if (!table.needSchedule(isLocalBalance)) { continue; @@ -1644,7 +1637,8 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me continue; } - for (Partition partition : globalStateMgr.getLocalMetastore().getAllPartitionsIncludeRecycleBin(olapTbl)) { + for (Partition partition : globalStateMgr.getStarRocksMeta() + .getAllPartitionsIncludeRecycleBin(olapTbl)) { partitionChecked++; if (partitionChecked % partitionBatchNum == 0) { lockTotalTime += System.nanoTime() - lockStart; @@ -1654,14 +1648,14 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me locker.lockDatabase(db.getId(), LockType.READ); LOG.debug("balancer get lock again"); lockStart = System.nanoTime(); - if (globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId) == null) { + if (globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId) == null) { continue DATABASE; } - if (globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, olapTbl.getId()) == null) { + if (globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, olapTbl.getId()) == null) { continue TABLE; } - if (globalStateMgr.getLocalMetastore().getPartitionIncludeRecycleBin(olapTbl, partition.getId()) == - null) { + if (globalStateMgr.getStarRocksMeta() + .getPartitionIncludeRecycleBin(olapTbl, partition.getId()) == null) { continue; } } @@ -1671,7 +1665,7 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me continue; } - DataProperty dataProperty = globalStateMgr.getLocalMetastore() + DataProperty dataProperty = globalStateMgr.getStarRocksMeta() .getDataPropertyIncludeRecycleBin(olapTbl.getPartitionInfo(), partition.getId()); if (dataProperty == null) { continue; @@ -1681,7 +1675,7 @@ private Map, PartitionStat> getPartitionStats(TStorageMedium me continue; } - int replicationFactor = globalStateMgr.getLocalMetastore() + int replicationFactor = globalStateMgr.getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTbl.getPartitionInfo(), partition.getId()); int replicaNum = partition.getDistributionInfo().getBucketNum() * replicationFactor; // replicaNum may be negative, cause getReplicationNumIncludeRecycleBin can return -1 @@ -1773,9 +1767,9 @@ private static Map getBackendOrPathToReplicaNum(List beIds, private Map getPartitionReplicaCnt() { GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); Map partitionReplicaCnt = new HashMap<>(); - List dbIds = globalStateMgr.getLocalMetastore().getDbIdsIncludeRecycleBin(); + List dbIds = globalStateMgr.getStarRocksMeta().getDbIdsIncludeRecycleBin(); for (Long dbId : dbIds) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -1787,7 +1781,7 @@ private Map getPartitionReplicaCnt() { Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.READ); try { - for (Table table : globalStateMgr.getLocalMetastore().getTablesIncludeRecycleBin(db)) { + for (Table table : globalStateMgr.getStarRocksMeta().getTablesIncludeRecycleBin(db)) { // check table is olap table or colocate table if (!table.needSchedule(false)) { continue; @@ -1798,11 +1792,12 @@ private Map getPartitionReplicaCnt() { } OlapTable olapTbl = (OlapTable) table; - for (Partition partition : globalStateMgr.getLocalMetastore().getAllPartitionsIncludeRecycleBin(olapTbl)) { + for (Partition partition : globalStateMgr.getStarRocksMeta() + .getAllPartitionsIncludeRecycleBin(olapTbl)) { int replicaTotalCnt = partition.getDistributionInfo().getBucketNum() * - globalStateMgr.getLocalMetastore().getReplicationNumIncludeRecycleBin(olapTbl.getPartitionInfo(), - partition.getId()); + globalStateMgr.getStarRocksMeta() + .getReplicationNumIncludeRecycleBin(olapTbl.getPartitionInfo(), partition.getId()); partitionReplicaCnt.put(partition.getId(), replicaTotalCnt); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/com/starrocks/clone/DynamicPartitionScheduler.java index 3cc2ad92d54a0..f9039004cb075 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/DynamicPartitionScheduler.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/DynamicPartitionScheduler.java @@ -352,6 +352,10 @@ private ArrayList getDropPartitionClause(Database db, OlapT Range checkDropPartitionKey = idToRange.getValue(); RangeUtils.checkRangeIntersect(reservePartitionKeyRange, checkDropPartitionKey); if (checkDropPartitionKey.upperEndpoint().compareTo(reservePartitionKeyRange.lowerEndpoint()) <= 0) { + + Partition partition = GlobalStateMgr.getCurrentState().getStarRocksMeta().getPartition( + db, olapTable, checkDropPartitionId); + String dropPartitionName = olapTable.getPartition(checkDropPartitionId).getName(); dropPartitionClauses.add(new DropPartitionClause(false, dropPartitionName, false, true)); } @@ -456,7 +460,7 @@ public boolean executeDynamicPartitionForTable(Long dbId, Long tableId) { AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(olapTable); analyzer.analyze(ctx, dropPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(db, olapTable, dropPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropPartition(db, olapTable, dropPartitionClause); clearDropPartitionFailedMsg(tableName); } catch (DdlException e) { recordDropPartitionFailedMsg(db.getOriginName(), tableName, e.getMessage()); @@ -471,7 +475,7 @@ public boolean executeDynamicPartitionForTable(Long dbId, Long tableId) { AlterTableClauseAnalyzer alterTableClauseVisitor = new AlterTableClauseAnalyzer(olapTable); alterTableClauseVisitor.analyze(ctx, addPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore().addPartitions(ctx, + GlobalStateMgr.getCurrentState().getStarRocksMeta().addPartitions(ctx, db, tableName, addPartitionClause); clearCreatePartitionFailedMsg(tableName); } catch (DdlException e) { @@ -551,7 +555,7 @@ private void scheduleTTLPartition() { new Locker(), db.getId(), Lists.newArrayList(olapTable.getId()), LockType.WRITE)) { AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(olapTable); analyzer.analyze(new ConnectContext(), dropPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(db, olapTable, dropPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropPartition(db, olapTable, dropPartitionClause); clearDropPartitionFailedMsg(tableName); } catch (DdlException e) { recordDropPartitionFailedMsg(db.getOriginName(), tableName, e.getMessage()); diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/TabletChecker.java b/fe/fe-core/src/main/java/com/starrocks/clone/TabletChecker.java index 622a90df3efa4..9e372fa5bc5dc 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/TabletChecker.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/TabletChecker.java @@ -267,10 +267,10 @@ private void doCheck(boolean isUrgent) { long lockTotalTime = 0; long waitTotalTime = 0; long lockStart; - List dbIds = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIdsIncludeRecycleBin(); + List dbIds = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIdsIncludeRecycleBin(); DATABASE: for (Long dbId : dbIds) { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -289,7 +289,7 @@ private void doCheck(boolean isUrgent) { List aliveBeIdsInCluster = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getBackendIds(true); TABLE: - for (Table table : GlobalStateMgr.getCurrentState().getLocalMetastore().getTablesIncludeRecycleBin(db)) { + for (Table table : GlobalStateMgr.getCurrentState().getStarRocksMeta().getTablesIncludeRecycleBin(db)) { if (!table.needSchedule(false)) { continue; } @@ -303,7 +303,7 @@ private void doCheck(boolean isUrgent) { } OlapTable olapTbl = (OlapTable) table; - for (Partition partition : GlobalStateMgr.getCurrentState().getLocalMetastore() + for (Partition partition : GlobalStateMgr.getCurrentState().getStarRocksMeta() .getAllPartitionsIncludeRecycleBin(olapTbl)) { partitionChecked++; @@ -321,15 +321,15 @@ private void doCheck(boolean isUrgent) { locker.lockDatabase(db.getId(), LockType.READ); LOG.debug("checker get lock again"); lockStart = System.nanoTime(); - if (GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(dbId) == null) { + if (GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(dbId) == null) { continue DATABASE; } - if (GlobalStateMgr.getCurrentState().getLocalMetastore() + if (GlobalStateMgr.getCurrentState().getStarRocksMeta() .getTableIncludeRecycleBin(db, olapTbl.getId()) == null) { continue TABLE; } if (GlobalStateMgr.getCurrentState() - .getLocalMetastore().getPartitionIncludeRecycleBin(olapTbl, partition.getId()) == null) { + .getStarRocksMeta().getPartitionIncludeRecycleBin(olapTbl, partition.getId()) == null) { continue; } } @@ -341,7 +341,7 @@ private void doCheck(boolean isUrgent) { } short replicaNum = GlobalStateMgr.getCurrentState() - .getLocalMetastore() + .getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTbl.getPartitionInfo(), partition.getId()); if (replicaNum == (short) -1) { continue; diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/com/starrocks/clone/TabletSchedCtx.java index 31573532334c4..8a4db80399b36 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/TabletSchedCtx.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/TabletSchedCtx.java @@ -60,6 +60,7 @@ import com.starrocks.common.util.TimeUtils; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ReplicaPersistInfo; import com.starrocks.privilege.AccessDeniedException; import com.starrocks.privilege.PrivilegeType; @@ -848,7 +849,7 @@ public CloneTask createCloneReplicaAndTask() throws SchedException { // if this is a balance task, or this is a repair task with REPLICA_MISSING/REPLICA_RELOCATING, // we create a new replica with state CLONE - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db " + dbId + " not exist"); } @@ -923,15 +924,15 @@ public CreateReplicaTask createEmptyReplicaAndTask() throws SchedException { tabletId, tablet.getSingleReplica(), tablet.getSingleReplica().getBackendId(), destBackendId); final GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db " + dbId + " does not exist"); } Locker locker = new Locker(); try { locker.lockDatabase(db.getId(), LockType.WRITE); - OlapTable olapTable = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin( - globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId), + OlapTable olapTable = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin( + globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId), tblId); if (olapTable == null) { throw new SchedException(Status.UNRECOVERABLE, "table " + tblId + " does not exist"); @@ -1010,6 +1011,7 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) Preconditions.checkArgument(cloneTask.getTaskVersion() == CloneTask.VERSION_2); setLastVisitedTime(System.currentTimeMillis()); GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); // check if clone task success if (request.getTask_status().getStatus_code() != TStatusCode.OK) { @@ -1035,32 +1037,33 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) } // 1. check the tablet status first - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db does not exist"); } Locker locker = new Locker(); try { locker.lockDatabase(db.getId(), LockType.WRITE); - OlapTable olapTable = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tblId); + OlapTable olapTable = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tblId); if (olapTable == null) { throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); } - PhysicalPartition partition = globalStateMgr.getLocalMetastore() + PhysicalPartition partition = globalStateMgr.getStarRocksMeta() .getPhysicalPartitionIncludeRecycleBin(olapTable, physicalPartitionId); if (partition == null) { throw new SchedException(Status.UNRECOVERABLE, "partition does not exist"); } short replicationNum = - globalStateMgr.getLocalMetastore() + globalStateMgr.getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), partitionId); if (replicationNum == (short) -1) { throw new SchedException(Status.UNRECOVERABLE, "invalid replication number"); } - MaterializedIndex index = partition.getIndex(indexId); + MaterializedIndex index = + GlobalStateMgr.getCurrentState().getTabletMetastore().getMaterializedIndex(partition, indexId); if (index == null) { throw new SchedException(Status.UNRECOVERABLE, "index does not exist"); } @@ -1071,7 +1074,7 @@ public void finishCloneTask(CloneTask cloneTask, TFinishTaskRequest request) + ", task's: " + schemaHash); } - LocalTablet tablet = (LocalTablet) index.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) tabletMetastore.getTablet(index, tabletId); if (tablet == null) { throw new SchedException(Status.UNRECOVERABLE, "tablet does not exist"); } @@ -1184,7 +1187,7 @@ private String unprotectedFinishClone(TFinishTaskRequest request, PhysicalPartit if (replica.getState() == ReplicaState.CLONE) { replica.setState(ReplicaState.NORMAL); tablet.setLastFullCloneFinishedTimeMs(System.currentTimeMillis()); - GlobalStateMgr.getCurrentState().getEditLog().logAddReplica(info); + GlobalStateMgr.getCurrentState().getTabletMetastore().addReplica(info, tablet, replica); } else { // if in VERSION_INCOMPLETE, replica is not newly created, thus the state is not CLONE // so, we keep it state unchanged, and log update replica diff --git a/fe/fe-core/src/main/java/com/starrocks/clone/TabletScheduler.java b/fe/fe-core/src/main/java/com/starrocks/clone/TabletScheduler.java index 283fe9204fca5..5f43974f5c4c3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/clone/TabletScheduler.java +++ b/fe/fe-core/src/main/java/com/starrocks/clone/TabletScheduler.java @@ -71,6 +71,7 @@ import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.leader.ReportHandler; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ReplicaPersistInfo; import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.ast.UserIdentity; @@ -690,8 +691,9 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) tabletCtx.setLastVisitedTime(currentTime); stat.counterTabletScheduled.incrementAndGet(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); // check this tablet again - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(tabletCtx.getDbId()); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(tabletCtx.getDbId()); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db does not exist"); } @@ -701,7 +703,7 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) locker.lockDatabase(db.getId(), LockType.READ); try { OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState() - .getLocalMetastore().getTableIncludeRecycleBin(db, tabletCtx.getTblId()); + .getStarRocksMeta().getTableIncludeRecycleBin(db, tabletCtx.getTblId()); if (tbl == null) { throw new SchedException(Status.UNRECOVERABLE, "tbl does not exist"); } @@ -714,24 +716,25 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) OlapTableState tableState = tbl.getState(); Partition partition = GlobalStateMgr.getCurrentState() - .getLocalMetastore().getPartitionIncludeRecycleBin(tbl, tabletCtx.getPartitionId()); + .getStarRocksMeta().getPartitionIncludeRecycleBin(tbl, tabletCtx.getPartitionId()); if (partition == null) { throw new SchedException(Status.UNRECOVERABLE, "partition does not exist"); } short replicaNum = GlobalStateMgr.getCurrentState() - .getLocalMetastore().getReplicationNumIncludeRecycleBin(tbl.getPartitionInfo(), partition.getId()); + .getStarRocksMeta().getReplicationNumIncludeRecycleBin(tbl.getPartitionInfo(), partition.getId()); if (replicaNum == (short) -1) { throw new SchedException(Status.UNRECOVERABLE, "invalid replication number"); } - DataProperty dataProperty = GlobalStateMgr.getCurrentState().getLocalMetastore() + DataProperty dataProperty = GlobalStateMgr.getCurrentState().getStarRocksMeta() .getDataPropertyIncludeRecycleBin(tbl.getPartitionInfo(), partition.getId()); if (dataProperty == null) { throw new SchedException(Status.UNRECOVERABLE, "partition data property not exist"); } - PhysicalPartition physicalPartition = partition.getSubPartition(tabletCtx.getPhysicalPartitionId()); + PhysicalPartition physicalPartition = GlobalStateMgr.getCurrentState().getTabletMetastore() + .getPhysicalPartition(partition, tabletCtx.getPhysicalPartitionId()); if (physicalPartition == null) { throw new SchedException(Status.UNRECOVERABLE, "physical partition " + tabletCtx.getPhysicalPartitionId() + "does not exist"); @@ -742,7 +745,7 @@ private void scheduleTablet(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) throw new SchedException(Status.UNRECOVERABLE, "index does not exist"); } - LocalTablet tablet = (LocalTablet) idx.getTablet(tabletCtx.getTabletId()); + LocalTablet tablet = (LocalTablet) tabletMetastore.getTablet(idx, tabletCtx.getTabletId()); Preconditions.checkNotNull(tablet); if (isColocateTable) { @@ -1067,7 +1070,7 @@ private void handleReplicaRelocating(TabletSchedCtx tabletCtx, AgentBatchTask ba private void handleRedundantReplica(TabletSchedCtx tabletCtx, boolean force) throws SchedException { stat.counterReplicaRedundantErr.incrementAndGet(); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(tabletCtx.getDbId()); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(tabletCtx.getDbId()); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db " + tabletCtx.getDbId() + " not exist"); } @@ -1301,7 +1304,7 @@ private boolean handleColocateRedundant(TabletSchedCtx tabletCtx) throws SchedEx Set backendSet = tabletCtx.getColocateBackendsSet(); Preconditions.checkNotNull(backendSet); stat.counterReplicaColocateRedundant.incrementAndGet(); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(tabletCtx.getDbId()); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(tabletCtx.getDbId()); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db " + tabletCtx.getDbId() + " not exist"); } @@ -1872,7 +1875,7 @@ public void finishCreateReplicaTask(CreateReplicaTask task, TFinishTaskRequest r replica.getSchemaHash(), replica.getDataSize(), replica.getRowCount(), replica.getLastFailedVersion(), replica.getLastSuccessVersion(), replica.getMinReadableVersion()); - GlobalStateMgr.getCurrentState().getEditLog().logAddReplica(info); + GlobalStateMgr.getCurrentState().getTabletMetastore().addReplica(info, tabletCtx.getTablet(), replica); finalizeTabletCtx(tabletCtx, TabletSchedCtx.State.FINISHED, "create replica finished"); LOG.info("create replica for recovery successfully, tablet:{} backend:{}", tabletId, task.getBackendId()); } @@ -2075,19 +2078,20 @@ public TGetTabletScheduleResponse getTabletSchedule(TGetTabletScheduleRequest re // caller should hold db lock private void checkMetaExist(TabletSchedCtx ctx) throws SchedException { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(ctx.getDbId()); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(ctx.getDbId()); if (db == null) { throw new SchedException(Status.UNRECOVERABLE, "db " + ctx.getDbId() + " dose not exist"); } OlapTable tbl = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTableIncludeRecycleBin(db, ctx.getTblId()); + (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getTableIncludeRecycleBin(db, ctx.getTblId()); if (tbl == null) { throw new SchedException(Status.UNRECOVERABLE, "table " + ctx.getTblId() + " dose not exist"); } - Partition partition = - GlobalStateMgr.getCurrentState().getLocalMetastore().getPartitionIncludeRecycleBin(tbl, ctx.getPartitionId()); + Partition partition = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getPartitionIncludeRecycleBin(tbl, ctx.getPartitionId()); if (partition == null) { throw new SchedException(Status.UNRECOVERABLE, "partition " + ctx.getPartitionId() + " dose not exist"); } @@ -2103,7 +2107,8 @@ private void checkMetaExist(TabletSchedCtx ctx) throws SchedException { throw new SchedException(Status.UNRECOVERABLE, "materialized index " + ctx.getIndexId() + " dose not exist"); } - Tablet tablet = idx.getTablet(ctx.getTabletId()); + Tablet tablet = GlobalStateMgr.getCurrentState().getTabletMetastore().getTablet(idx, ctx.getTabletId()); + if (tablet == null) { throw new SchedException(Status.UNRECOVERABLE, "tablet " + ctx.getTabletId() + " dose not exist"); } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/proc/LakeTabletsProcDir.java b/fe/fe-core/src/main/java/com/starrocks/common/proc/LakeTabletsProcDir.java index 5004e62b9d553..9c54d2f2bb87a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/proc/LakeTabletsProcDir.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/proc/LakeTabletsProcDir.java @@ -30,6 +30,7 @@ import com.starrocks.lake.LakeTablet; import com.starrocks.monitor.unit.ByteSizeValue; import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; import java.util.Arrays; import java.util.Collections; @@ -135,7 +136,7 @@ public ProcNodeInterface lookup(String tabletIdStr) throws AnalysisException { Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.READ); try { - Tablet tablet = index.getTablet(tabletId); + Tablet tablet = GlobalStateMgr.getCurrentState().getTabletMetastore().getTablet(index, tabletId); if (tablet == null) { throw new AnalysisException("Can't find tablet id: " + tabletIdStr); } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java index dd8c55a5888ce..54f0eecb219cf 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java @@ -1564,7 +1564,7 @@ public static void analyzeMVProperties(Database db, StorageVolumeMgr svm = GlobalStateMgr.getCurrentState().getStorageVolumeMgr(); svm.bindTableToStorageVolume(volume, db.getId(), materializedView.getId()); String storageVolumeId = svm.getStorageVolumeIdOfTable(materializedView.getId()); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .setLakeStorageInfo(db, materializedView, storageVolumeId, properties); } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/AutoCloseableLock.java b/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/AutoCloseableLock.java index 0498ea1f81aac..e737ceab77a36 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/AutoCloseableLock.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/AutoCloseableLock.java @@ -22,6 +22,15 @@ public class AutoCloseableLock implements AutoCloseable { private final List tableList; private final LockType lockType; + public AutoCloseableLock(Locker locker, Long dbId, LockType lockType) { + this.locker = locker; + this.dbId = dbId; + this.tableList = new ArrayList<>(); + this.lockType = lockType; + + locker.lockTablesWithIntensiveDbLock(dbId, tableList, lockType); + } + public AutoCloseableLock(Locker locker, Long dbId, List tableList, LockType lockType) { this.locker = locker; this.dbId = dbId; @@ -31,6 +40,7 @@ public AutoCloseableLock(Locker locker, Long dbId, List tableList, LockTyp locker.lockTablesWithIntensiveDbLock(dbId, tableList, lockType); } + @Override public void close() { locker.unLockTablesWithIntensiveDbLock(dbId, tableList, lockType); diff --git a/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/Locker.java b/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/Locker.java index 2528874e1210b..551df4a00bc6b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/Locker.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/util/concurrent/lock/Locker.java @@ -133,6 +133,37 @@ public void lockDatabase(Long dbId, LockType lockType) { } } + public void lockDatabaseWithIntendLock(Database database, LockType lockType) { + if (Config.lock_manager_enabled) { + Preconditions.checkState(database != null); + try { + lock(database.getId(), lockType, 0); + } catch (LockException e) { + throw ErrorReportException.report(ErrorCode.ERR_LOCK_ERROR, e.getMessage()); + } + } else { + QueryableReentrantReadWriteLock rwLock = database.getRwLock(); + if (lockType == LockType.INTENTION_EXCLUSIVE) { + LockUtils.dbWriteLock(rwLock, database.getId(), database.getFullName(), database.getSlowLockLogStats()); + } else { + LockUtils.dbReadLock(rwLock, database.getId(), database.getFullName(), database.getSlowLockLogStats()); + } + } + } + + public void lockTable(Table table, LockType lockType) { + if (Config.lock_manager_enabled) { + Preconditions.checkState(table != null); + try { + lock(table.getId(), lockType, 0); + } catch (LockException e) { + throw ErrorReportException.report(ErrorCode.ERR_LOCK_ERROR, e.getMessage()); + } + } else { + // + } + } + /** * Before the new version of LockManager is fully enabled, it is used to be compatible with the original db lock logic. */ @@ -433,7 +464,7 @@ public void unLockTableWithIntensiveDbLock(Long dbId, Long tableId, LockType loc /** * Lock table with intensive db lock. * - * @param dbId db for intensive db lock + * @param dbId db for intensive db lock * @param tableId table to be locked * @param lockType lock type */ diff --git a/fe/fe-core/src/main/java/com/starrocks/consistency/CheckConsistencyJob.java b/fe/fe-core/src/main/java/com/starrocks/consistency/CheckConsistencyJob.java index d7cc2f8357967..496d9cd2a346b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/consistency/CheckConsistencyJob.java +++ b/fe/fe-core/src/main/java/com/starrocks/consistency/CheckConsistencyJob.java @@ -52,6 +52,7 @@ import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.journal.JournalTask; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ConsistencyCheckInfo; import com.starrocks.persist.EditLog; import com.starrocks.server.GlobalStateMgr; @@ -122,6 +123,7 @@ public long getTabletId() { */ public boolean sendTasks() { TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId); if (tabletMeta == null) { LOG.debug("tablet[{}] has been removed", tabletId); @@ -166,7 +168,7 @@ public boolean sendTasks() { return false; } - tablet = (LocalTablet) index.getTablet(tabletId); + tablet = (LocalTablet) tabletMetastore.getTablet(index, tabletId); if (tablet == null) { LOG.debug("tablet[{}] does not exist", tabletId); return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/com/starrocks/consistency/ConsistencyChecker.java index 8afdd1812b35d..451adc369cab2 100644 --- a/fe/fe-core/src/main/java/com/starrocks/consistency/ConsistencyChecker.java +++ b/fe/fe-core/src/main/java/com/starrocks/consistency/ConsistencyChecker.java @@ -58,6 +58,7 @@ import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.consistency.CheckConsistencyJob.JobState; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ConsistencyCheckInfo; import com.starrocks.server.GlobalStateMgr; import com.starrocks.server.LocalMetastore; @@ -165,6 +166,7 @@ private void checkTabletMetaConsistency(Map creatingTableIds) { LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); CatalogRecycleBin recycleBin = GlobalStateMgr.getCurrentState().getRecycleBin(); TabletInvertedIndex tabletInvertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); Set invalidTablets = new HashSet<>(); // backend id -> @@ -264,7 +266,7 @@ private void checkTabletMetaConsistency(Map creatingTableIds) { if (!table.isCloudNativeTableOrMaterializedView()) { // validate tablet - Tablet tablet = index.getTablet(tabletId); + Tablet tablet = tabletMetastore.getTablet(index, tabletId); if (tablet == null) { deleteTabletByConsistencyChecker(tabletMeta, tabletId, backendId, "tablet " + dbId + "." + tableId + "." + @@ -614,7 +616,8 @@ public void replayFinishConsistencyCheck(ConsistencyCheckInfo info, GlobalStateM LOG.warn("replay finish consistency check failed, index is null, info: {}", info); return; } - LocalTablet tablet = (LocalTablet) index.getTablet(info.getTabletId()); + LocalTablet tablet = (LocalTablet) GlobalStateMgr.getCurrentState() + .getTabletMetastore().getTablet(index, info.getTabletId()); if (tablet == null) { LOG.warn("replay finish consistency check failed, tablet is null, info: {}", info); return; diff --git a/fe/fe-core/src/main/java/com/starrocks/http/meta/ColocateMetaService.java b/fe/fe-core/src/main/java/com/starrocks/http/meta/ColocateMetaService.java index 23407e2a58e8d..973d1b968bafd 100644 --- a/fe/fe-core/src/main/java/com/starrocks/http/meta/ColocateMetaService.java +++ b/fe/fe-core/src/main/java/com/starrocks/http/meta/ColocateMetaService.java @@ -262,7 +262,7 @@ public void executeInLeaderWithAdmin(BaseRequest request, BaseResponse response) isJoin = false; } - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(groupId.dbId); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(groupId.dbId); if (db == null) { response.appendContent("Non-exist db"); writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); @@ -271,7 +271,7 @@ public void executeInLeaderWithAdmin(BaseRequest request, BaseResponse response) Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.WRITE); try { - OlapTable table = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable table = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tableId); if (table == null) { response.appendContent("Non-exist table"); writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); diff --git a/fe/fe-core/src/main/java/com/starrocks/http/meta/GlobalDictMetaService.java b/fe/fe-core/src/main/java/com/starrocks/http/meta/GlobalDictMetaService.java index 74c34acdf5987..c29421642724a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/http/meta/GlobalDictMetaService.java +++ b/fe/fe-core/src/main/java/com/starrocks/http/meta/GlobalDictMetaService.java @@ -16,22 +16,33 @@ package com.starrocks.http.meta; import com.google.common.base.Strings; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Table; import com.starrocks.common.DdlException; +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.common.util.concurrent.lock.LockType; +import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.http.ActionController; import com.starrocks.http.BaseRequest; import com.starrocks.http.BaseResponse; import com.starrocks.http.IllegalArgException; import com.starrocks.http.rest.RestBaseAction; import com.starrocks.http.rest.RestBaseResult; +import com.starrocks.persist.ModifyTablePropertyOperationLog; import com.starrocks.privilege.AccessDeniedException; import com.starrocks.qe.ConnectContext; import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.ast.UserIdentity; +import com.starrocks.sql.optimizer.statistics.IDictManager; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpResponseStatus; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.HashMap; +import java.util.Map; + /** * eg: * POST /api/global_dict/table/enable?db_name=test&table_name=test_basic&enable=false @@ -98,8 +109,7 @@ public void executeInLeaderWithAdmin(BaseRequest request, BaseResponse response) return; } boolean isEnable = Boolean.parseBoolean(enableParam.trim()); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .setHasForbiddenGlobalDict(dbName, tableName, isEnable); + setHasForbiddenGlobalDict(dbName, tableName, isEnable); response.appendContent(new RestBaseResult("apply success").toJson()); } else { response.appendContent(new RestBaseResult("HTTP method is not allowed.").toJson()); @@ -108,5 +118,38 @@ public void executeInLeaderWithAdmin(BaseRequest request, BaseResponse response) } sendResult(request, response); } + + public void setHasForbiddenGlobalDict(String dbName, String tableName, boolean isForbit) throws DdlException { + Map property = new HashMap<>(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbName); + if (db == null) { + throw new DdlException("the DB " + dbName + " is not exist"); + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), tableName); + if (table == null) { + throw new DdlException("the DB " + dbName + " table: " + tableName + "isn't exist"); + } + + if (table instanceof OlapTable) { + OlapTable olapTable = (OlapTable) table; + olapTable.setHasForbiddenGlobalDict(isForbit); + if (isForbit) { + property.put(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE, PropertyAnalyzer.DISABLE_LOW_CARD_DICT); + IDictManager.getInstance().disableGlobalDict(olapTable.getId()); + } else { + property.put(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE, PropertyAnalyzer.ABLE_LOW_CARD_DICT); + IDictManager.getInstance().enableGlobalDict(olapTable.getId()); + } + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), property); + GlobalStateMgr.getCurrentState().getEditLog().logSetHasForbiddenGlobalDict(info); + } + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/http/rest/MigrationAction.java b/fe/fe-core/src/main/java/com/starrocks/http/rest/MigrationAction.java index 0b27df6bd9fb0..80aa8993cce3f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/http/rest/MigrationAction.java +++ b/fe/fe-core/src/main/java/com/starrocks/http/rest/MigrationAction.java @@ -41,6 +41,7 @@ import com.starrocks.catalog.MaterializedIndex; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; import com.starrocks.catalog.Replica; import com.starrocks.catalog.Table; import com.starrocks.catalog.Table.TableType; @@ -118,19 +119,21 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response OlapTable olapTable = (OlapTable) table; for (Partition partition : olapTable.getPartitions()) { - String partitionName = partition.getName(); - MaterializedIndex baseIndex = partition.getBaseIndex(); - for (Tablet tablet : baseIndex.getTablets()) { - List row = Lists.newArrayList(); - row.add(tableName); - row.add(partitionName); - row.add(tablet.getId()); - row.add(olapTable.getSchemaHashByIndexId(baseIndex.getId())); - if (CollectionUtils.isNotEmpty(((LocalTablet) tablet).getImmutableReplicas())) { - Replica replica = ((LocalTablet) tablet).getImmutableReplicas().get(0); - row.add(replica.getBackendId()); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + String partitionName = physicalPartition.getName(); + MaterializedIndex baseIndex = physicalPartition.getBaseIndex(); + for (Tablet tablet : baseIndex.getTablets()) { + List row = Lists.newArrayList(); + row.add(tableName); + row.add(partitionName); + row.add(tablet.getId()); + row.add(olapTable.getSchemaHashByIndexId(baseIndex.getId())); + if (CollectionUtils.isNotEmpty(((LocalTablet) tablet).getImmutableReplicas())) { + Replica replica = ((LocalTablet) tablet).getImmutableReplicas().get(0); + row.add(replica.getBackendId()); + } + rows.add(row); } - rows.add(row); } } } else { @@ -144,19 +147,21 @@ protected void executeWithoutPassword(BaseRequest request, BaseResponse response tableName = table.getName(); for (Partition partition : olapTable.getPartitions()) { - String partitionName = partition.getName(); - MaterializedIndex baseIndex = partition.getBaseIndex(); - for (Tablet tablet : baseIndex.getTablets()) { - List row = Lists.newArrayList(); - row.add(tableName); - row.add(partitionName); - row.add(tablet.getId()); - row.add(olapTable.getSchemaHashByIndexId(baseIndex.getId())); - if (CollectionUtils.isNotEmpty(((LocalTablet) tablet).getImmutableReplicas())) { - Replica replica = ((LocalTablet) tablet).getImmutableReplicas().get(0); - row.add(replica.getBackendId()); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + String partitionName = physicalPartition.getName(); + MaterializedIndex baseIndex = physicalPartition.getBaseIndex(); + for (Tablet tablet : baseIndex.getTablets()) { + List row = Lists.newArrayList(); + row.add(tableName); + row.add(partitionName); + row.add(tablet.getId()); + row.add(olapTable.getSchemaHashByIndexId(baseIndex.getId())); + if (CollectionUtils.isNotEmpty(((LocalTablet) tablet).getImmutableReplicas())) { + Replica replica = ((LocalTablet) tablet).getImmutableReplicas().get(0); + row.add(replica.getBackendId()); + } + rows.add(row); } - rows.add(row); } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/journal/JournalWriter.java b/fe/fe-core/src/main/java/com/starrocks/journal/JournalWriter.java index fde313bf0fcf2..5cc332f9eefe8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/journal/JournalWriter.java +++ b/fe/fe-core/src/main/java/com/starrocks/journal/JournalWriter.java @@ -16,12 +16,19 @@ package com.starrocks.journal; import com.starrocks.common.Config; +import com.starrocks.common.io.Text; +import com.starrocks.common.io.Writable; import com.starrocks.common.util.Daemon; import com.starrocks.common.util.Util; import com.starrocks.metric.MetricRepo; +import com.starrocks.persist.DatabaseInfo; +import com.starrocks.persist.OperationType; +import com.starrocks.persist.gson.GsonUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; import java.util.ArrayList; import java.util.List; import java.util.concurrent.BlockingQueue; @@ -105,6 +112,20 @@ protected void writeOneBatch() throws InterruptedException { long nextJournalId = nextVisibleJournalId; initBatch(); + JournalEntity journalEntity = new JournalEntity(); + DataInputStream in = new DataInputStream(new ByteArrayInputStream(currentJournal.getBuffer().getData())); + try { + journalEntity.readFields(in); + if (journalEntity.getOpCode() == OperationType.OP_RENAME_DB_V2) { + Writable data = GsonUtils.GSON.fromJson(Text.readString(in), DatabaseInfo.class); + DatabaseInfo databaseInfo = (DatabaseInfo) data; + + //databaseInfo.transaction; + } + } catch (Exception e) { + + } + try { this.journal.batchWriteBegin(); diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeListPartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeListPartitionInfo.java index 29cfcfaa45636..f0509d24b55ea 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeListPartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeListPartitionInfo.java @@ -39,7 +39,7 @@ public boolean delete() { WarehouseManager manager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); Warehouse warehouse = manager.getBackgroundWarehouse(); if (LakeTableHelper.removePartitionDirectory(partition, warehouse.getId())) { - GlobalStateMgr.getCurrentState().getLocalMetastore().onErasePartition(partition); + GlobalStateMgr.getCurrentState().getStarRocksMeta().onErasePartition(partition); return true; } else { return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeRangePartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeRangePartitionInfo.java index c5a5e43e65efa..087233ac21bda 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeRangePartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeRangePartitionInfo.java @@ -42,7 +42,7 @@ public boolean delete() { WarehouseManager manager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); Warehouse warehouse = manager.getBackgroundWarehouse(); if (LakeTableHelper.removePartitionDirectory(partition, warehouse.getId())) { - GlobalStateMgr.getCurrentState().getLocalMetastore().onErasePartition(partition); + GlobalStateMgr.getCurrentState().getStarRocksMeta().onErasePartition(partition); return true; } else { return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeUnPartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeUnPartitionInfo.java index 1366381271f19..3ef5fc29bf53e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeUnPartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/RecycleLakeUnPartitionInfo.java @@ -41,7 +41,7 @@ public boolean delete() { WarehouseManager manager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); Warehouse warehouse = manager.getBackgroundWarehouse(); if (LakeTableHelper.removePartitionDirectory(partition, warehouse.getId())) { - GlobalStateMgr.getCurrentState().getLocalMetastore().onErasePartition(partition); + GlobalStateMgr.getCurrentState().getStarRocksMeta().onErasePartition(partition); return true; } else { return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/lake/StarMgrMetaSyncer.java b/fe/fe-core/src/main/java/com/starrocks/lake/StarMgrMetaSyncer.java index 9d57c9fe14dc0..3c819cb87fede 100644 --- a/fe/fe-core/src/main/java/com/starrocks/lake/StarMgrMetaSyncer.java +++ b/fe/fe-core/src/main/java/com/starrocks/lake/StarMgrMetaSyncer.java @@ -63,9 +63,9 @@ public StarMgrMetaSyncer() { private List getAllPartitionShardGroupId() { List groupIds = new ArrayList<>(); - List dbIds = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIdsIncludeRecycleBin(); + List dbIds = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIdsIncludeRecycleBin(); for (Long dbId : dbIds) { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -76,9 +76,9 @@ private List getAllPartitionShardGroupId() { Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.READ); try { - for (Table table : GlobalStateMgr.getCurrentState().getLocalMetastore().getTablesIncludeRecycleBin(db)) { + for (Table table : GlobalStateMgr.getCurrentState().getStarRocksMeta().getTablesIncludeRecycleBin(db)) { if (table.isCloudNativeTableOrMaterializedView()) { - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .getAllPartitionsIncludeRecycleBin((OlapTable) table) .stream() .map(Partition::getSubPartitions) @@ -296,7 +296,7 @@ public boolean syncTableMetaInternal(Database db, OlapTable table, boolean force if (GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getId(), table.getId()) == null) { return false; // table might be dropped } - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .getAllPartitionsIncludeRecycleBin(table) .stream() .map(Partition::getSubPartitions) diff --git a/fe/fe-core/src/main/java/com/starrocks/leader/ReportHandler.java b/fe/fe-core/src/main/java/com/starrocks/leader/ReportHandler.java index 24c4f0b6b0833..db82d47e2c708 100644 --- a/fe/fe-core/src/main/java/com/starrocks/leader/ReportHandler.java +++ b/fe/fe-core/src/main/java/com/starrocks/leader/ReportHandler.java @@ -38,10 +38,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashBasedTable; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import com.google.common.collect.Queues; import com.google.common.collect.Sets; import com.google.common.collect.Table; @@ -49,6 +51,7 @@ import com.starrocks.catalog.ColocateTableIndex; import com.starrocks.catalog.Column; import com.starrocks.catalog.ColumnId; +import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; import com.starrocks.catalog.DiskInfo; import com.starrocks.catalog.KeysType; @@ -59,6 +62,7 @@ import com.starrocks.catalog.MaterializedIndexMeta; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; import com.starrocks.catalog.PhysicalPartition; import com.starrocks.catalog.Replica; import com.starrocks.catalog.Replica.ReplicaState; @@ -79,11 +83,14 @@ import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.datacache.DataCacheMetrics; import com.starrocks.memory.MemoryTrackable; +import com.starrocks.meta.StarRocksMeta; +import com.starrocks.meta.TabletMetastore; import com.starrocks.metric.GaugeMetric; import com.starrocks.metric.Metric.MetricUnit; import com.starrocks.metric.MetricRepo; import com.starrocks.persist.BackendTabletsInfo; import com.starrocks.persist.BatchDeleteReplicaInfo; +import com.starrocks.persist.ModifyPartitionInfo; import com.starrocks.persist.ReplicaPersistInfo; import com.starrocks.server.GlobalStateMgr; import com.starrocks.server.RunMode; @@ -135,6 +142,7 @@ import org.apache.thrift.TException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -142,6 +150,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; public class ReportHandler extends Daemon implements MemoryTrackable { @@ -459,8 +468,7 @@ private static void tabletReport(long backendId, Map backendTable backendId, backendTablets.size(), backendReportVersion); // storage medium map - HashMap storageMediumMap = - GlobalStateMgr.getCurrentState().getLocalMetastore().getPartitionIdToStorageMediumMap(); + HashMap storageMediumMap = getPartitionIdToStorageMediumMap(); // db id -> tablet id ListMultimap tabletSyncMap = ArrayListMultimap.create(); @@ -739,6 +747,123 @@ public static void tabletReport(long backendId, Map backendTablet tabletMigrationMap.size(), transactionsToClear.size(), transactionsToPublish.size(), (end - start)); } + public static HashMap getPartitionIdToStorageMediumMap() { + StarRocksMeta starRocksMeta = GlobalStateMgr.getCurrentState().getStarRocksMeta(); + HashMap storageMediumMap = new HashMap<>(); + + // record partition which need to change storage medium + // dbId -> (tableId -> partitionId) + HashMap> changedPartitionsMap = new HashMap<>(); + long currentTimeMs = System.currentTimeMillis(); + List dbIds = starRocksMeta.getDbIds(); + + for (long dbId : dbIds) { + Database db = starRocksMeta.getDb(dbId); + if (db == null) { + LOG.warn("db {} does not exist while doing backend report", dbId); + continue; + } + + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + for (com.starrocks.catalog.Table table : db.getTables()) { + if (!table.isOlapTableOrMaterializedView()) { + continue; + } + + long tableId = table.getId(); + OlapTable olapTable = (OlapTable) table; + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + for (Partition partition : olapTable.getAllPartitions()) { + long partitionId = partition.getId(); + DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); + Preconditions.checkNotNull(dataProperty, + partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); + // only normal state table can migrate. + // PRIMARY_KEYS table does not support local migration. + if (dataProperty.getStorageMedium() == TStorageMedium.SSD + && dataProperty.getCooldownTimeMs() < currentTimeMs + && olapTable.getState() == OlapTable.OlapTableState.NORMAL) { + // expire. change to HDD. + // record and change when holding write lock + Multimap multimap = changedPartitionsMap.get(dbId); + if (multimap == null) { + multimap = HashMultimap.create(); + changedPartitionsMap.put(dbId, multimap); + } + multimap.put(tableId, partitionId); + } else { + storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); + } + } // end for partitions + } // end for tables + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + } // end for dbs + + // handle data property changed + for (Long dbId : changedPartitionsMap.keySet()) { + Database db = starRocksMeta.getDb(dbId); + if (db == null) { + LOG.warn("db {} does not exist while checking backend storage medium", dbId); + continue; + } + Multimap tableIdToPartitionIds = changedPartitionsMap.get(dbId); + + // use try lock to avoid blocking a long time. + // if block too long, backend report rpc will timeout. + Locker locker = new Locker(); + if (!locker.tryLockDatabase(db.getId(), LockType.WRITE, Database.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + LOG.warn("try get db {}-{} write lock but failed when checking backend storage medium", + db.getFullName(), dbId); + continue; + } + Preconditions.checkState(locker.isDbWriteLockHeldByCurrentThread(db)); + try { + for (Long tableId : tableIdToPartitionIds.keySet()) { + com.starrocks.catalog.Table table = starRocksMeta.getTable(db.getId(), tableId); + if (table == null) { + continue; + } + OlapTable olapTable = (OlapTable) table; + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + + Collection partitionIds = tableIdToPartitionIds.get(tableId); + for (Long partitionId : partitionIds) { + Partition partition = olapTable.getPartition(partitionId); + if (partition == null) { + continue; + } + DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); + if (dataProperty.getStorageMedium() == TStorageMedium.SSD + && dataProperty.getCooldownTimeMs() < currentTimeMs) { + // expire. change to HDD. + DataProperty hdd = new DataProperty(TStorageMedium.HDD); + partitionInfo.setDataProperty(partition.getId(), hdd); + storageMediumMap.put(partitionId, TStorageMedium.HDD); + LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD", + dbId, tableId, partitionId); + + // log + ModifyPartitionInfo info = + new ModifyPartitionInfo(db.getId(), olapTable.getId(), + partition.getId(), + hdd, + (short) -1, + partitionInfo.getIsInMemory(partition.getId())); + GlobalStateMgr.getCurrentState().getEditLog().logModifyPartition(info); + } + } // end for partitions + } // end for tables + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } // end for dbs + return storageMediumMap; + } + private static boolean needSync(Replica replicaInFe, TTabletInfo backendTabletInfo) { if (backendTabletInfo.isSetUsed() && !backendTabletInfo.isUsed()) { // tablet is bad, do not sync @@ -912,9 +1037,10 @@ private static void datacacheMetricsReport(long backendId, TDataCacheMetrics met private static void sync(Map backendTablets, ListMultimap tabletSyncMap, long backendId, long backendReportVersion) { TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); for (Long dbId : tabletSyncMap.keySet()) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -945,12 +1071,12 @@ private static void sync(Map backendTablets, ListMultimap backendTablets, ListMultimap tabletDeleteFromMeta long backendReportVersion) { AgentBatchTask createReplicaBatchTask = new AgentBatchTask(); TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); Map hashToDiskInfo = new HashMap<>(); for (DiskInfo diskInfo : GlobalStateMgr.getCurrentState().getNodeMgr() @@ -1076,7 +1204,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta List replicaPersistInfoList = new ArrayList<>(); DB_TRAVERSE: for (Long dbId : tabletDeleteFromMeta.keySet()) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue; } @@ -1094,7 +1222,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta long currentTime = System.currentTimeMillis(); if (currentTime - lockStartTime > MAX_DB_WLOCK_HOLDING_TIME_MS) { locker.unLockDatabase(db.getId(), LockType.WRITE); - db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { continue DB_TRAVERSE; } @@ -1113,25 +1241,25 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta LOG.debug("delete tablet {} in partition {} of table {} in db {} from meta. backend[{}]", tabletId, partitionId, tableId, dbId, backendId); - OlapTable olapTable = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable olapTable = (OlapTable) globalStateMgr.getStarRocksMeta() + .getTableIncludeRecycleBin(db, tableId); if (olapTable == null) { continue; } - if (globalStateMgr.getLocalMetastore() + if (globalStateMgr.getStarRocksMeta() .getPartitionIncludeRecycleBin(olapTable, tabletMeta.getPartitionId()) == null) { continue; } - PhysicalPartition partition = globalStateMgr.getLocalMetastore() + PhysicalPartition partition = globalStateMgr.getStarRocksMeta() .getPhysicalPartitionIncludeRecycleBin(olapTable, partitionId); if (partition == null) { continue; } - short replicationNum = - globalStateMgr.getLocalMetastore().getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), - tabletMeta.getPartitionId()); + short replicationNum = globalStateMgr.getStarRocksMeta() + .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), tabletMeta.getPartitionId()); if (replicationNum == (short) -1) { continue; } @@ -1147,7 +1275,7 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta continue; } - LocalTablet tablet = (LocalTablet) index.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) tabletMetastore.getTablet(index, tabletId); if (tablet == null) { continue; } @@ -1571,6 +1699,7 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM tabletRecoveryMap.size(), backendId); TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); BackendTabletsInfo backendTabletsInfo = new BackendTabletsInfo(backendId); backendTabletsInfo.setBad(true); for (Long dbId : tabletRecoveryMap.keySet()) { @@ -1610,7 +1739,7 @@ private static void handleRecoverTablet(ListMultimap tabletRecoveryM int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - LocalTablet tablet = (LocalTablet) index.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) tabletMetastore.getTablet(index, tabletId); if (tablet == null) { continue; } @@ -2038,6 +2167,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); SystemInfoService infoService = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); TabletMeta tabletMeta = invertedIndex.getTabletMeta(tabletId); long dbId = tabletMeta != null ? tabletMeta.getDbId() : TabletInvertedIndex.NOT_EXIST_VALUE; @@ -2052,28 +2182,28 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon long dataSize = backendTabletInfo.getData_size(); long rowCount = backendTabletInfo.getRow_count(); - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(dbId); if (db == null) { throw new MetaNotFoundException("db[" + dbId + "] does not exist"); } - OlapTable olapTable = (OlapTable) globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, tableId); + OlapTable olapTable = (OlapTable) globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, tableId); if (olapTable == null) { throw new MetaNotFoundException("table[" + tableId + "] does not exist"); } Locker locker = new Locker(); locker.lockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(olapTable.getId()), LockType.WRITE); try { - if (globalStateMgr.getLocalMetastore().getPartitionIncludeRecycleBin(olapTable, partitionId) == null) { + if (globalStateMgr.getStarRocksMeta().getPartitionIncludeRecycleBin(olapTable, partitionId) == null) { throw new MetaNotFoundException("partition[" + partitionId + "] does not exist"); } short replicationNum = - globalStateMgr.getLocalMetastore() + globalStateMgr.getStarRocksMeta() .getReplicationNumIncludeRecycleBin(olapTable.getPartitionInfo(), partitionId); if (replicationNum == (short) -1) { throw new MetaNotFoundException("invalid replication number of partition [" + partitionId + "]"); } - PhysicalPartition partition = globalStateMgr.getLocalMetastore() + PhysicalPartition partition = globalStateMgr.getStarRocksMeta() .getPhysicalPartitionIncludeRecycleBin(olapTable, physicalPartitionId); if (partition == null) { throw new MetaNotFoundException("physical partition[" + physicalPartitionId + "] does not exist"); @@ -2084,7 +2214,8 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon throw new MetaNotFoundException("index[" + indexId + "] does not exist"); } - LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(tabletId); + LocalTablet tablet = (LocalTablet) GlobalStateMgr.getCurrentState() + .getTabletMetastore().getTablet(materializedIndex, tabletId); if (tablet == null) { throw new MetaNotFoundException("tablet[" + tabletId + "] does not exist"); } @@ -2151,7 +2282,6 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon Replica replica = new Replica(replicaId, backendId, version, schemaHash, dataSize, rowCount, ReplicaState.NORMAL, lastFailedVersion, version); - tablet.addReplica(replica); // write edit log ReplicaPersistInfo info = ReplicaPersistInfo.createForAdd(dbId, tableId, physicalPartitionId, indexId, @@ -2159,7 +2289,7 @@ private static void addReplica(long tabletId, TTabletInfo backendTabletInfo, lon version, schemaHash, dataSize, rowCount, lastFailedVersion, version, minReadableVersion); - GlobalStateMgr.getCurrentState().getEditLog().logAddReplica(info); + tabletMetastore.addReplica(info, tablet, replica); LOG.info("add replica[{}-{}] to globalStateMgr. backend:[{}] replicas: {}", tabletId, replicaId, backendId, tablet.getReplicaInfos()); diff --git a/fe/fe-core/src/main/java/com/starrocks/listener/LoadJobMVListener.java b/fe/fe-core/src/main/java/com/starrocks/listener/LoadJobMVListener.java index 36547a351ecce..d3b44f128d0d6 100644 --- a/fe/fe-core/src/main/java/com/starrocks/listener/LoadJobMVListener.java +++ b/fe/fe-core/src/main/java/com/starrocks/listener/LoadJobMVListener.java @@ -153,7 +153,7 @@ private void doTriggerToRefreshRelatedMVs(Database db, Table table) throws DdlEx LOG.info("Trigger auto materialized view refresh because of base table {} has changed, " + "db:{}, mv:{}", table.getName(), mvDb.getFullName(), materializedView.getName()); - GlobalStateMgr.getCurrentState().getLocalMetastore().refreshMaterializedView( + GlobalStateMgr.getCurrentState().getStarRocksMeta().refreshMaterializedView( mvDb.getFullName(), materializedView.getName(), false, null, Constants.TaskRunPriority.NORMAL.value(), true, false); } diff --git a/fe/fe-core/src/main/java/com/starrocks/load/InsertOverwriteJobRunner.java b/fe/fe-core/src/main/java/com/starrocks/load/InsertOverwriteJobRunner.java index d40c4565ffaff..80d19d6029eaa 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/InsertOverwriteJobRunner.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/InsertOverwriteJobRunner.java @@ -297,7 +297,7 @@ private void createPartitionByValue(InsertStmt insertStmt) { try { AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(olapTable); analyzer.analyze(context, addPartitionClause); - state.getLocalMetastore().addPartitions(context, db, olapTable.getName(), addPartitionClause); + state.getStarRocksMeta().addPartitions(context, db, olapTable.getName(), addPartitionClause); } catch (Exception ex) { LOG.warn(ex.getMessage(), ex); throw new RuntimeException(ex); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/PartitionUtils.java b/fe/fe-core/src/main/java/com/starrocks/load/PartitionUtils.java index fdeec97534be4..2a6d50534d79d 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/PartitionUtils.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/PartitionUtils.java @@ -56,7 +56,7 @@ public static void createAndAddTempPartitionsForTable(Database db, OlapTable tar List tmpPartitionIds, DistributionDesc distributionDesc, long warehouseId) throws DdlException { - List newTempPartitions = GlobalStateMgr.getCurrentState().getLocalMetastore() + List newTempPartitions = GlobalStateMgr.getCurrentState().getStarRocksMeta() .createTempPartitionsFromPartitions(db, targetTable, postfix, sourcePartitionIds, tmpPartitionIds, distributionDesc, warehouseId); Locker locker = new Locker(); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadsHistorySyncer.java b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadsHistorySyncer.java index 3f7cbad563491..37b2354ee7ce0 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadsHistorySyncer.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/loadv2/LoadsHistorySyncer.java @@ -98,7 +98,7 @@ public static void createTable() throws UserException { public static boolean correctTable() { int numBackends = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getTotalBackendNumber(); - int replica = GlobalStateMgr.getCurrentState().getLocalMetastore() + int replica = GlobalStateMgr.getCurrentState().getStarRocksMeta() .mayGetTable(LOADS_HISTORY_DB_NAME, LOADS_HISTORY_TABLE_NAME) .map(tbl -> ((OlapTable) tbl).getPartitionInfo().getMinReplicationNum()) .orElse((short) 1); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/pipe/Pipe.java b/fe/fe-core/src/main/java/com/starrocks/load/pipe/Pipe.java index 834a1fd543350..9f3fd8e802151 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/pipe/Pipe.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/pipe/Pipe.java @@ -314,7 +314,7 @@ private void buildNewTasks() { long taskId = GlobalStateMgr.getCurrentState().getNextId(); PipeId pipeId = getPipeId(); String uniqueName = PipeTaskDesc.genUniqueTaskName(getName(), taskId, 0); - String dbName = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(pipeId.getDbId()) + String dbName = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(pipeId.getDbId()) .map(Database::getOriginName) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_BAD_DB_ERROR)); String sqlTask = FilePipeSource.buildInsertSql(this, piece, uniqueName); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/pipe/PipeManager.java b/fe/fe-core/src/main/java/com/starrocks/load/pipe/PipeManager.java index f17b8cc9f25e5..10008bbc2729e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/pipe/PipeManager.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/pipe/PipeManager.java @@ -184,7 +184,7 @@ protected void updatePipe(Pipe pipe) { } private Pair resolvePipeNameUnlock(PipeName name) { - long dbId = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(name.getDbName()) + long dbId = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(name.getDbName()) .map(Database::getId) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_NO_DB_ERROR)); return Pair.create(dbId, name.getPipeName()); diff --git a/fe/fe-core/src/main/java/com/starrocks/load/pipe/filelist/RepoCreator.java b/fe/fe-core/src/main/java/com/starrocks/load/pipe/filelist/RepoCreator.java index aea6240a646a2..6742c97ad238a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/load/pipe/filelist/RepoCreator.java +++ b/fe/fe-core/src/main/java/com/starrocks/load/pipe/filelist/RepoCreator.java @@ -71,7 +71,7 @@ public static void createTable() throws UserException { public static boolean correctTable() { int numBackends = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getTotalBackendNumber(); int replica = GlobalStateMgr.getCurrentState() - .getLocalMetastore().mayGetTable(FileListTableRepo.FILE_LIST_DB_NAME, FileListTableRepo.FILE_LIST_TABLE_NAME) + .getStarRocksMeta().mayGetTable(FileListTableRepo.FILE_LIST_DB_NAME, FileListTableRepo.FILE_LIST_TABLE_NAME) .map(tbl -> ((OlapTable) tbl).getPartitionInfo().getMinReplicationNum()) .orElse((short) 1); if (numBackends < 3) { diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/BDBDatabase.java b/fe/fe-core/src/main/java/com/starrocks/meta/BDBDatabase.java new file mode 100644 index 0000000000000..2d56f26eee81b --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/BDBDatabase.java @@ -0,0 +1,33 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.sleepycat.je.Database; +import com.sleepycat.je.DatabaseConfig; +import com.sleepycat.je.Environment; + +public class BDBDatabase { + public static Database openDatabase(Environment environment, String db) { + DatabaseConfig databaseConfig = new DatabaseConfig(); + databaseConfig.setTransactional(true); + databaseConfig.setAllowCreate(true); + databaseConfig.setReadOnly(false); + databaseConfig.setSortedDuplicatesVoid(false); + return environment.openDatabase(null, db, databaseConfig); + } + + public static void truncateDatabase(Environment environment, String db) { + environment.truncateDatabase(null, db, false); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/BDBTransaction.java b/fe/fe-core/src/main/java/com/starrocks/meta/BDBTransaction.java new file mode 100644 index 0000000000000..1d548f59fd456 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/BDBTransaction.java @@ -0,0 +1,33 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.sleepycat.je.Durability; +import com.sleepycat.je.Environment; +import com.sleepycat.je.Transaction; +import com.sleepycat.je.TransactionConfig; + +import java.util.concurrent.TimeUnit; + +public class BDBTransaction { + public static Transaction startTransaction(Environment environment) { + TransactionConfig transactionConfig = new TransactionConfig(); + transactionConfig.setTxnTimeout(500, TimeUnit.SECONDS); + transactionConfig.setDurability(new Durability(Durability.SyncPolicy.SYNC, + Durability.SyncPolicy.SYNC, Durability.ReplicaAckPolicy.ALL)); + + return environment.beginTransaction(null, transactionConfig); + } +} + diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/ByteCoder.java b/fe/fe-core/src/main/java/com/starrocks/meta/ByteCoder.java new file mode 100644 index 0000000000000..cf580fb51038d --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/ByteCoder.java @@ -0,0 +1,36 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.starrocks.meta; + +import com.google.common.base.Joiner; +import org.apache.hadoop.util.Lists; + +import java.nio.charset.StandardCharsets; +import java.util.List; + +public class ByteCoder { + public static byte[] encode(List params) { + return Joiner.on("/").join(params).getBytes(StandardCharsets.UTF_8); + } + + public static byte[] encode(String... params) { + return Joiner.on("/").join(params).getBytes(StandardCharsets.UTF_8); + } + + public static List decode(byte[] params) { + String s = new String(params, StandardCharsets.UTF_8); + return Lists.newArrayList(s.split("/")); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/EditLogCommitter.java b/fe/fe-core/src/main/java/com/starrocks/meta/EditLogCommitter.java new file mode 100644 index 0000000000000..53dd2b6b98b46 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/EditLogCommitter.java @@ -0,0 +1,27 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.google.common.collect.Lists; +import com.starrocks.persist.OperationType; + +import java.util.List; + +public class EditLogCommitter { + public List ops = Lists.newArrayList( + OperationType.OP_CREATE_TABLE_V2 + ); + + +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreInterface.java b/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreInterface.java new file mode 100644 index 0000000000000..d0d23056f23d3 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreInterface.java @@ -0,0 +1,100 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.starrocks.meta; + +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.Table; +import com.starrocks.common.DdlException; +import com.starrocks.persist.CreateTableInfo; +import com.starrocks.persist.DatabaseInfo; +import com.starrocks.persist.DropInfo; +import com.starrocks.persist.DropPartitionInfo; +import com.starrocks.persist.DropPartitionsInfo; +import com.starrocks.persist.ModifyTablePropertyOperationLog; +import com.starrocks.persist.PartitionVersionRecoveryInfo; +import com.starrocks.persist.ReplacePartitionOperationLog; +import com.starrocks.persist.TableInfo; +import com.starrocks.persist.TruncateTableInfo; +import com.starrocks.sql.ast.PartitionDesc; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public interface LocalMetastoreInterface { + void createDb(Database db, String storageVolumeId); + + void dropDb(Database db, boolean isForceDrop); + + void recoverDatabase(Database db); + + void alterDatabaseQuota(DatabaseInfo dbInfo); + + void renameDatabase(String dbName, String newDbName); + + List listDbNames(); + + ConcurrentHashMap getIdToDb(); + + List getDbIds(); + + ConcurrentHashMap getFullNameToDb(); + + Database getDb(String name); + + Database getDb(long dbId); + + void createTable(CreateTableInfo createTableInfo); + + void dropTable(DropInfo dropInfo); + + void renameTable(TableInfo tableInfo); + + void truncateTable(TruncateTableInfo info); + + void alterTable(ModifyTablePropertyOperationLog log); + + void modifyTableProperty(Database db, OlapTable table, Map properties, short operationType); + + List listTableNames(String dbName); + + List getTables(Long dbId); + + Table getTable(String dbName, String tblName); + + Table getTable(Long dbId, Long tableId); + + void addPartitionLog(Database db, OlapTable olapTable, List partitionDescs, + boolean isTempPartition, PartitionInfo partitionInfo, + List partitionList, Set existPartitionNameSet) throws DdlException; + + void dropPartition(DropPartitionInfo dropPartitionInfo); + + void dropPartitions(DropPartitionsInfo dropPartitionsInfo); + + void renamePartition(TableInfo tableInfo); + + void replaceTempPartition(ReplacePartitionOperationLog info); + + void setPartitionVersion(PartitionVersionRecoveryInfo info); + + void addSubPartitionLog(Database db, OlapTable olapTable, Partition partition, List subPartitions); +} + diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreV2.java b/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreV2.java new file mode 100644 index 0000000000000..de52c26f9f4df --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/LocalMetastoreV2.java @@ -0,0 +1,231 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.sleepycat.je.Transaction; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.Table; +import com.starrocks.common.DdlException; +import com.starrocks.persist.CreateDbInfo; +import com.starrocks.persist.CreateTableInfo; +import com.starrocks.persist.DatabaseInfo; +import com.starrocks.persist.DropInfo; +import com.starrocks.persist.DropPartitionInfo; +import com.starrocks.persist.DropPartitionsInfo; +import com.starrocks.persist.ModifyTablePropertyOperationLog; +import com.starrocks.persist.OperationType; +import com.starrocks.persist.PartitionVersionRecoveryInfo; +import com.starrocks.persist.ReplacePartitionOperationLog; +import com.starrocks.persist.TableInfo; +import com.starrocks.persist.TruncateTableInfo; +import com.starrocks.persist.gson.GsonUtils; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.PartitionDesc; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class LocalMetastoreV2 implements LocalMetastoreInterface { + @Override + public void createDb(Database db, String storageVolumeId) { + MetadataHandler metadataHandler = GlobalStateMgr.getCurrentState().getMetadataHandler(); + Transaction transaction = metadataHandler.starTransaction(); + metadataHandler.put(transaction, + ByteCoder.encode("meta object", "db", String.valueOf(db.getId())), + GsonUtils.GSON.toJson(db, Database.class)); + metadataHandler.put(transaction, + ByteCoder.encode("meta name", "db", db.getFullName()), + String.valueOf(db.getId())); + metadataHandler.put(transaction, + ByteCoder.encode("meta id", "instance",String.valueOf(db.getId())), + ""); + + CreateDbInfo createDbInfo = new CreateDbInfo(db.getId(), db.getFullName()); + createDbInfo.setStorageVolumeId(storageVolumeId); + createDbInfo.setTransaction(transaction); + GlobalStateMgr.getCurrentState().getEditLog().logJsonObject(OperationType.OP_CREATE_DB_V2, createDbInfo); + } + + @Override + public void dropDb(Database db, boolean isForceDrop) { + + } + + @Override + public void recoverDatabase(Database db) { + + } + + @Override + public void alterDatabaseQuota(DatabaseInfo dbInfo) { + + } + + @Override + public void renameDatabase(String dbName, String newDbName) { + + } + + @Override + public List listDbNames() { + return null; + } + + @Override + public ConcurrentHashMap getIdToDb() { + return null; + } + + @Override + public List getDbIds() { + return null; + } + + @Override + public ConcurrentHashMap getFullNameToDb() { + return null; + } + + @Override + public Database getDb(String name) { + MetadataHandler metadataHandler = GlobalStateMgr.getCurrentState().getMetadataHandler(); + Transaction transaction = metadataHandler.starTransaction(); + Long databaseId = metadataHandler.get(transaction, ByteCoder.encode("meta name", "db", name), Long.class); + + String value = metadataHandler.get(transaction, + ByteCoder.encode("meta object", "db", String.valueOf(databaseId)), String.class); + + Database database = GsonUtils.GSON.fromJson(value, Database.class); + return database; + } + + @Override + public Database getDb(long dbId) { + return null; + } + + @Override + public void createTable(CreateTableInfo createTableInfo) { + + } + + @Override + public void dropTable(DropInfo dropInfo) { + + } + + @Override + public void renameTable(TableInfo tableInfo) { + + } + + @Override + public void truncateTable(TruncateTableInfo info) { + + } + + @Override + public void alterTable(ModifyTablePropertyOperationLog log) { + + } + + @Override + public void modifyTableProperty(Database db, OlapTable table, Map properties, short operationType) { + MetadataHandler metadataHandler = GlobalStateMgr.getCurrentState().getMetadataHandler(); + Transaction transaction = metadataHandler.starTransaction(); + metadataHandler.put(transaction, + ByteCoder.encode("meta object", "table", String.valueOf(table.getId())), + GsonUtils.GSON.toJson(table, OlapTable.class)); + + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + info.setTransaction(transaction); + + switch (operationType) { + case OperationType.OP_MODIFY_IN_MEMORY: + GlobalStateMgr.getCurrentState().getEditLog().logModifyInMemory(info); + break; + case OperationType.OP_MODIFY_WRITE_QUORUM: + GlobalStateMgr.getCurrentState().getEditLog().logModifyWriteQuorum(info); + break; + } + } + + @Override + public List listTableNames(String dbName) { + return null; + } + + @Override + public List
getTables(Long dbId) { + return null; + } + + @Override + public Table getTable(String dbName, String tblName) { + return null; + } + + @Override + public Table getTable(Long dbId, Long tableId) { + return null; + } + + @Override + public void addPartitionLog(Database db, + OlapTable olapTable, + List partitionDescs, + boolean isTempPartition, + PartitionInfo partitionInfo, + List partitionList, + Set existPartitionNameSet) throws DdlException { + + } + + @Override + public void dropPartition(DropPartitionInfo dropPartitionInfo) { + + } + + @Override + public void dropPartitions(DropPartitionsInfo dropPartitionsInfo) { + + } + + @Override + public void renamePartition(TableInfo tableInfo) { + + } + + @Override + public void replaceTempPartition(ReplacePartitionOperationLog info) { + + } + + @Override + public void setPartitionVersion(PartitionVersionRecoveryInfo info) { + + } + + @Override + public void addSubPartitionLog(Database db, OlapTable olapTable, Partition partition, List subPartitions) { + + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/MetadataHandler.java b/fe/fe-core/src/main/java/com/starrocks/meta/MetadataHandler.java new file mode 100644 index 0000000000000..fc287e02cae64 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/MetadataHandler.java @@ -0,0 +1,139 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.sleepycat.bind.tuple.TupleBinding; +import com.sleepycat.je.Cursor; +import com.sleepycat.je.Database; +import com.sleepycat.je.DatabaseConfig; +import com.sleepycat.je.DatabaseEntry; +import com.sleepycat.je.Durability; +import com.sleepycat.je.LockMode; +import com.sleepycat.je.OperationStatus; +import com.sleepycat.je.Transaction; +import com.sleepycat.je.TransactionConfig; +import com.sleepycat.je.dbi.DupKeyData; +import com.starrocks.journal.bdbje.BDBEnvironment; +import com.starrocks.server.GlobalStateMgr; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class MetadataHandler { + public static MetadataHandler getInstance() { + return GlobalStateMgr.getCurrentState().getMetadataHandler(); + } + + private final BDBEnvironment bdbEnvironment; + private final Database database; + + public MetadataHandler(BDBEnvironment bdbEnvironment) { + this.bdbEnvironment = bdbEnvironment; + + DatabaseConfig dbConfig = new DatabaseConfig(); + dbConfig.setTransactional(true); + if (GlobalStateMgr.getCurrentState().isElectable()) { + dbConfig.setAllowCreate(true); + dbConfig.setReadOnly(false); + } else { + dbConfig.setAllowCreate(false); + dbConfig.setReadOnly(true); + } + this.database = bdbEnvironment.getReplicatedEnvironment().openDatabase(null, "meta", dbConfig); + } + + public Transaction starTransaction() { + TransactionConfig transactionConfig = new TransactionConfig(); + transactionConfig.setTxnTimeout(500, TimeUnit.SECONDS); + transactionConfig.setDurability(new Durability(Durability.SyncPolicy.SYNC, + Durability.SyncPolicy.SYNC, Durability.ReplicaAckPolicy.ALL)); + + return bdbEnvironment.getReplicatedEnvironment().beginTransaction(null, transactionConfig); + } + + public OperationStatus put(Transaction transaction, String keyS, String valueS) { + TupleBinding binding = TupleBinding.getPrimitiveBinding(String.class); + DatabaseEntry key = new DatabaseEntry(); + binding.objectToEntry(keyS, key); + + DatabaseEntry value = new DatabaseEntry(); + binding.objectToEntry(valueS, value); + return database.put(transaction, key, value); + } + + public OperationStatus put(Transaction transaction, byte[] keyS, String valueS) { + TupleBinding binding = TupleBinding.getPrimitiveBinding(String.class); + DatabaseEntry key = new DatabaseEntry(keyS); + DatabaseEntry value = new DatabaseEntry(); + binding.objectToEntry(valueS, value); + return database.put(transaction, key, value); + } + + public byte[] get(Transaction transaction, byte[] keyS) { + TupleBinding binding = TupleBinding.getPrimitiveBinding(String.class); + DatabaseEntry key = new DatabaseEntry(keyS); + DatabaseEntry result = new DatabaseEntry(); + database.get(transaction, key, result, LockMode.READ_COMMITTED); + + binding.entryToObject(result); + return result.getData(); + } + + public T get(Transaction transaction, byte[] key, Class c) { + TupleBinding binding = TupleBinding.getPrimitiveBinding(c); + DatabaseEntry databaseEntry = new DatabaseEntry(key); + DatabaseEntry result = new DatabaseEntry(); + OperationStatus status = database.get(transaction, databaseEntry, result, LockMode.READ_COMMITTED); + if (status.equals(OperationStatus.NOTFOUND)) { + return null; + } else { + return binding.entryToObject(result); + } + } + + public List getPrefix(Transaction transaction, byte[] prefix) { + Cursor cursor = database.openCursor(transaction, null); + + DatabaseEntry key = new DatabaseEntry(prefix); + + DatabaseEntry prefixStart = new DatabaseEntry(prefix); + DatabaseEntry prefixEnd = new DatabaseEntry(DupKeyData.makePrefixKey(key.getData(), key.getOffset(), key.getSize())); + + DatabaseEntry noReturnData = new DatabaseEntry(); + noReturnData.setPartial(0, 0, true); + cursor.getSearchKeyRange(key, noReturnData, LockMode.READ_COMMITTED); + + List keyList = new ArrayList<>(); + do { + if (DupKeyData.compareMainKey( + key.getData(), + prefixEnd.getData(), + prefixEnd.getOffset(), + prefixEnd.getSize(), + database.getConfig().getBtreeComparator()) == 0) { + break; + } + + keyList.add(key.getData()); + + } while (cursor.getNext(key, noReturnData, LockMode.READ_COMMITTED) == OperationStatus.SUCCESS); + + return keyList; + } + + public OperationStatus delete(Transaction transaction, byte[] keyS) { + return null; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/ReplicaHierarchyId.java b/fe/fe-core/src/main/java/com/starrocks/meta/ReplicaHierarchyId.java new file mode 100644 index 0000000000000..0d3953a2fdda8 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/ReplicaHierarchyId.java @@ -0,0 +1,40 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +public class ReplicaHierarchyId { + public final long dbId; + public final long tableId; + public final long partitionId; + public final long physicalPartitionId; + public final long materializedIndexId; + public final long tabletId; + public final long replicaId; + + public ReplicaHierarchyId(long dbId, + long tableId, + long partitionId, + long physicalPartitionId, + long materializedIndexId, + long tabletId, + long replicaId) { + this.dbId = dbId; + this.tableId = tableId; + this.partitionId = partitionId; + this.physicalPartitionId = physicalPartitionId; + this.materializedIndexId = materializedIndexId; + this.tabletId = tabletId; + this.replicaId = replicaId; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/StarRocksMeta.java b/fe/fe-core/src/main/java/com/starrocks/meta/StarRocksMeta.java new file mode 100644 index 0000000000000..2f03951f2fa80 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/StarRocksMeta.java @@ -0,0 +1,2580 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import com.staros.proto.FilePathInfo; +import com.starrocks.alter.AlterJobExecutor; +import com.starrocks.alter.AlterMVJobExecutor; +import com.starrocks.alter.MaterializedViewHandler; +import com.starrocks.analysis.Expr; +import com.starrocks.analysis.FunctionCallExpr; +import com.starrocks.analysis.HintNode; +import com.starrocks.analysis.IntLiteral; +import com.starrocks.analysis.SetVarHint; +import com.starrocks.analysis.SlotRef; +import com.starrocks.analysis.StringLiteral; +import com.starrocks.analysis.TableName; +import com.starrocks.analysis.TableRef; +import com.starrocks.analysis.UserVariableHint; +import com.starrocks.catalog.CatalogRecycleBin; +import com.starrocks.catalog.CatalogUtils; +import com.starrocks.catalog.ColocateGroupSchema; +import com.starrocks.catalog.Column; +import com.starrocks.catalog.DataProperty; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.DistributionInfo; +import com.starrocks.catalog.FunctionSet; +import com.starrocks.catalog.HashDistributionInfo; +import com.starrocks.catalog.Index; +import com.starrocks.catalog.KeysType; +import com.starrocks.catalog.ListPartitionInfo; +import com.starrocks.catalog.LocalTablet; +import com.starrocks.catalog.MaterializedIndex; +import com.starrocks.catalog.MaterializedIndexMeta; +import com.starrocks.catalog.MaterializedView; +import com.starrocks.catalog.MvId; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PartitionKey; +import com.starrocks.catalog.PartitionType; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.PrimitiveType; +import com.starrocks.catalog.RandomDistributionInfo; +import com.starrocks.catalog.RangePartitionInfo; +import com.starrocks.catalog.Replica; +import com.starrocks.catalog.SinglePartitionInfo; +import com.starrocks.catalog.Table; +import com.starrocks.catalog.TableProperty; +import com.starrocks.catalog.Tablet; +import com.starrocks.catalog.TabletInvertedIndex; +import com.starrocks.catalog.TabletMeta; +import com.starrocks.catalog.View; +import com.starrocks.common.AlreadyExistsException; +import com.starrocks.common.AnalysisException; +import com.starrocks.common.Config; +import com.starrocks.common.DdlException; +import com.starrocks.common.ErrorCode; +import com.starrocks.common.ErrorReport; +import com.starrocks.common.ErrorReportException; +import com.starrocks.common.InvalidOlapTableStateException; +import com.starrocks.common.MaterializedViewExceptions; +import com.starrocks.common.MetaNotFoundException; +import com.starrocks.common.Pair; +import com.starrocks.common.UserException; +import com.starrocks.common.util.DynamicPartitionUtil; +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.common.util.TimeUtils; +import com.starrocks.common.util.Util; +import com.starrocks.common.util.concurrent.lock.LockType; +import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.connector.ConnectorMetadata; +import com.starrocks.lake.DataCacheInfo; +import com.starrocks.lake.LakeMaterializedView; +import com.starrocks.lake.LakeTable; +import com.starrocks.lake.StorageInfo; +import com.starrocks.load.pipe.PipeManager; +import com.starrocks.mv.analyzer.MVPartitionExprResolver; +import com.starrocks.persist.CreateTableInfo; +import com.starrocks.persist.DatabaseInfo; +import com.starrocks.persist.DropPartitionInfo; +import com.starrocks.persist.DropPartitionsInfo; +import com.starrocks.persist.ModifyTablePropertyOperationLog; +import com.starrocks.persist.PartitionVersionRecoveryInfo; +import com.starrocks.persist.ReplacePartitionOperationLog; +import com.starrocks.persist.TableInfo; +import com.starrocks.persist.TruncateTableInfo; +import com.starrocks.privilege.AccessDeniedException; +import com.starrocks.privilege.ObjectType; +import com.starrocks.privilege.PrivilegeType; +import com.starrocks.qe.ConnectContext; +import com.starrocks.qe.SessionVariable; +import com.starrocks.qe.VariableMgr; +import com.starrocks.scheduler.Constants; +import com.starrocks.scheduler.ExecuteOption; +import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskBuilder; +import com.starrocks.scheduler.TaskManager; +import com.starrocks.scheduler.TaskRun; +import com.starrocks.server.AbstractTableFactory; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.server.LocalMetastore; +import com.starrocks.server.RunMode; +import com.starrocks.server.StorageVolumeMgr; +import com.starrocks.server.TableFactoryProvider; +import com.starrocks.sql.analyzer.AnalyzerUtils; +import com.starrocks.sql.analyzer.Authorizer; +import com.starrocks.sql.ast.AddPartitionClause; +import com.starrocks.sql.ast.AdminSetPartitionVersionStmt; +import com.starrocks.sql.ast.AlterDatabaseQuotaStmt; +import com.starrocks.sql.ast.AlterDatabaseRenameStatement; +import com.starrocks.sql.ast.AlterMaterializedViewStmt; +import com.starrocks.sql.ast.AlterTableCommentClause; +import com.starrocks.sql.ast.AlterTableStmt; +import com.starrocks.sql.ast.AlterViewStmt; +import com.starrocks.sql.ast.AsyncRefreshSchemeDesc; +import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; +import com.starrocks.sql.ast.CreateMaterializedViewStatement; +import com.starrocks.sql.ast.CreateMaterializedViewStmt; +import com.starrocks.sql.ast.CreateTableLikeStmt; +import com.starrocks.sql.ast.CreateTableStmt; +import com.starrocks.sql.ast.CreateTemporaryTableStmt; +import com.starrocks.sql.ast.CreateViewStmt; +import com.starrocks.sql.ast.DistributionDesc; +import com.starrocks.sql.ast.DropMaterializedViewStmt; +import com.starrocks.sql.ast.DropPartitionClause; +import com.starrocks.sql.ast.DropTableStmt; +import com.starrocks.sql.ast.ExpressionPartitionDesc; +import com.starrocks.sql.ast.IntervalLiteral; +import com.starrocks.sql.ast.PartitionDesc; +import com.starrocks.sql.ast.PartitionRangeDesc; +import com.starrocks.sql.ast.PartitionRenameClause; +import com.starrocks.sql.ast.RecoverDbStmt; +import com.starrocks.sql.ast.RecoverPartitionStmt; +import com.starrocks.sql.ast.RecoverTableStmt; +import com.starrocks.sql.ast.RefreshMaterializedViewStatement; +import com.starrocks.sql.ast.RefreshSchemeClause; +import com.starrocks.sql.ast.ReplacePartitionClause; +import com.starrocks.sql.ast.SystemVariable; +import com.starrocks.sql.ast.TableRenameClause; +import com.starrocks.sql.ast.TruncateTableStmt; +import com.starrocks.sql.common.MetaUtils; +import com.starrocks.sql.common.SyncPartitionUtils; +import com.starrocks.sql.optimizer.Utils; +import com.starrocks.task.TabletTaskExecutor; +import com.starrocks.thrift.TGetTasksParams; +import com.starrocks.thrift.TStorageMedium; +import com.starrocks.thrift.TStorageType; +import com.starrocks.thrift.TTabletType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class StarRocksMeta implements ConnectorMetadata { + private static final Logger LOG = LogManager.getLogger(StarRocksMeta.class); + + @Override + public void createDb(String dbName, Map properties) throws DdlException, AlreadyExistsException { + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); + + long id = 0L; + if (!globalStateMgr.tryLock(false)) { + throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); + } + try { + if (localMetastore.getDb(dbName) != null) { + throw new AlreadyExistsException("Database Already Exists"); + } else { + id = globalStateMgr.getNextId(); + Database db = new Database(id, dbName); + String volume = StorageVolumeMgr.DEFAULT; + if (properties != null && properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VOLUME)) { + volume = properties.remove(PropertyAnalyzer.PROPERTIES_STORAGE_VOLUME); + } + if (!GlobalStateMgr.getCurrentState().getStorageVolumeMgr().bindDbToStorageVolume(volume, id)) { + throw new DdlException(String.format("Storage volume %s not exists", volume)); + } + GlobalStateMgr.getCurrentState().getGlobalTransactionMgr().addDatabaseTransactionMgr(db.getId()); + String storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr().getStorageVolumeIdOfDb(id); + + localMetastore.createDb(db, storageVolumeId); + } + } finally { + globalStateMgr.unlock(); + } + LOG.info("createDb dbName = " + dbName + ", id = " + id); + } + + @Override + public void dropDb(String dbName, boolean isForceDrop) throws DdlException, MetaNotFoundException { + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); + CatalogRecycleBin recycleBin = GlobalStateMgr.getCurrentState().getRecycleBin(); + + // 1. check if database exists + Database db; + if (!globalStateMgr.tryLock(false)) { + throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); + } + try { + db = localMetastore.getDb(dbName); + if (db == null) { + throw new MetaNotFoundException("Database not found"); + } + if (!isForceDrop && !db.getTemporaryTables().isEmpty()) { + throw new DdlException("The database [" + dbName + "] " + + "cannot be dropped because there are still some temporary tables in it. " + + "If you want to forcibly drop, please use \"DROP DATABASE FORCE.\""); + } + } finally { + globalStateMgr.unlock(); + } + + // 2. drop tables in db + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + if (!db.isExist()) { + throw new MetaNotFoundException("Database '" + dbName + "' not found"); + } + if (!isForceDrop && GlobalStateMgr.getCurrentState() + .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\"."); + } + + // 3. remove db from globalStateMgr + // save table names for recycling + Set tableNames = new HashSet<>(db.getTableNamesViewWithLock()); + localMetastore.dropDb(db, isForceDrop); + recycleBin.recycleDatabase(db, tableNames, isForceDrop); + db.setExist(false); + + // 4. drop mv task + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + TGetTasksParams tasksParams = new TGetTasksParams(); + tasksParams.setDb(dbName); + List dropTaskIdList = taskManager.filterTasks(tasksParams) + .stream().map(Task::getId).collect(Collectors.toList()); + taskManager.dropTasks(dropTaskIdList, false); + + // 5. Drop Pipes + PipeManager pipeManager = GlobalStateMgr.getCurrentState().getPipeManager(); + pipeManager.dropPipesOfDb(dbName, db.getId()); + + LOG.info("finish drop database[{}], id: {}, is force : {}", dbName, db.getId(), isForceDrop); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void recoverDatabase(RecoverDbStmt recoverStmt) throws DdlException { + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); + CatalogRecycleBin recycleBin = GlobalStateMgr.getCurrentState().getRecycleBin(); + + // check is new db with same name already exist + if (getDb(recoverStmt.getDbName()) != null) { + throw new DdlException("Database[" + recoverStmt.getDbName() + "] already exist."); + } + + Database db = recycleBin.recoverDatabase(recoverStmt.getDbName()); + + // add db to globalStateMgr + if (!globalStateMgr.tryLock(false)) { + throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); + } + try { + if (localMetastore.getDb(db.getFullName()) != null) { + throw new DdlException("Database[" + db.getOriginName() + "] already exist."); + // it's ok that we do not put db back to CatalogRecycleBin + // cause this db cannot recover anymore + } + + List materializedViews = db.getMaterializedViews(); + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + for (MaterializedView materializedView : materializedViews) { + MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); + if (refreshType != MaterializedView.RefreshType.SYNC) { + Task task = TaskBuilder.buildMvTask(materializedView, db.getFullName()); + TaskBuilder.updateTaskInfo(task, materializedView); + taskManager.createTask(task, false); + } + } + + localMetastore.recoverDatabase(db); + } finally { + globalStateMgr.unlock(); + } + + LOG.info("finish recover database, name: {}, id: {}", recoverStmt.getDbName(), db.getId()); + } + + public void alterDatabaseQuota(AlterDatabaseQuotaStmt stmt) throws DdlException { + LocalMetastore localMetastore = GlobalStateMgr.getCurrentState().getLocalMetastore(); + + String dbName = stmt.getDbName(); + Database db = getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + + DatabaseInfo dbInfo = new DatabaseInfo(db.getFullName(), "", stmt.getQuota(), stmt.getQuotaType()); + localMetastore.alterDatabaseQuota(dbInfo); + } + + public void renameDatabase(AlterDatabaseRenameStatement stmt) throws DdlException { + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + LocalMetastore localMetastore = globalStateMgr.getLocalMetastore(); + + String fullDbName = stmt.getDbName(); + String newFullDbName = stmt.getNewDbName(); + + if (fullDbName.equals(newFullDbName)) { + throw new DdlException("Same database name"); + } + + + if (!globalStateMgr.tryLock(false)) { + throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); + } + try { + // check if db exists + Database db = localMetastore.getDb(fullDbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, fullDbName); + } + + // check if name is already used + if (localMetastore.getDb(newFullDbName) != null) { + throw new DdlException("Database name[" + newFullDbName + "] is already used"); + } + + localMetastore.renameDatabase(fullDbName, newFullDbName); + } finally { + globalStateMgr.unlock(); + } + + LOG.info("rename database[{}] to [{}], id: {}", fullDbName, newFullDbName, 0); + } + + @Override + public List listDbNames() { + return GlobalStateMgr.getCurrentState().getLocalMetastore().listDbNames(); + } + + @Override + public Database getDb(String name) { + return GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(name); + } + + @Override + public Database getDb(long dbId) { + return GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); + } + + public Optional mayGetDb(String name) { + return Optional.ofNullable(getDb(name)); + } + + public Optional mayGetDb(long dbId) { + return Optional.ofNullable(getDb(dbId)); + } + + /** + * Following is the step to create an olap table: + * 1. create columns + * 2. create partition info + * 3. create distribution info + * 4. set table id and base index id + * 5. set bloom filter columns + * 6. set and build TableProperty includes: + * 6.1. dynamicProperty + * 6.2. replicationNum + * 6.3. inMemory + * 7. set index meta + * 8. check colocation properties + * 9. create tablet in BE + * 10. add this table to FE's meta + * 11. add this table to ColocateGroup if necessary + * + * @return whether the table is created + */ + @Override + public boolean createTable(CreateTableStmt stmt) throws DdlException { + // check if db exists + Database db = getDb(stmt.getDbName()); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDbName()); + } + + boolean isTemporaryTable = (stmt instanceof CreateTemporaryTableStmt); + // perform the existence check which is cheap before any further heavy operations. + // NOTE: don't even check the quota if already exists. + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + String tableName = stmt.getTableName(); + if (!isTemporaryTable && getTable(db.getFullName(), tableName) != null) { + if (!stmt.isSetIfNotExists()) { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } + LOG.info("create table[{}] which already exists", tableName); + return false; + } + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + + // only internal table should check quota and cluster capacity + if (!stmt.isExternal()) { + // check cluster capacity + GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().checkClusterCapacity(); + // check db quota + db.checkQuota(); + } + + AbstractTableFactory tableFactory = TableFactoryProvider.getFactory(stmt.getEngineName()); + if (tableFactory == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_STORAGE_ENGINE, stmt.getEngineName()); + } + + Table table = tableFactory.createTable(GlobalStateMgr.getCurrentState().getLocalMetastore(), db, stmt); + String storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr() + .getStorageVolumeIdOfTable(table.getId()); + + try { + onCreate(db, table, storageVolumeId, stmt.isSetIfNotExists()); + } catch (DdlException e) { + if (table.isCloudNativeTable()) { + GlobalStateMgr.getCurrentState().getStorageVolumeMgr().unbindTableToStorageVolume(table.getId()); + } + throw e; + } + return true; + } + + public void onCreate(Database db, Table table, String storageVolumeId, boolean isSetIfNotExists) + throws DdlException { + // check database exists again, because database can be dropped when creating table + if (!GlobalStateMgr.getCurrentState().tryLock(false)) { + throw new DdlException("Failed to acquire globalStateMgr lock. " + + "Try again or increasing value of `catalog_try_lock_timeout_ms` configuration."); + } + + try { + /* + * When creating table or mv, we need to create the tablets and prepare some of the + * metadata first before putting this new table or mv in the database. So after the + * first step, we need to acquire the global lock and double check whether the db still + * exists because it maybe dropped by other concurrent client. And if we don't use the lock + * protection and handle the concurrency properly, the replay of table/mv creation may fail + * on restart or on follower. + * + * After acquire the db lock, we also need to acquire the db lock and write edit log. Since the + * db lock maybe under high contention and IO is busy, current thread can hold the global lock + * for quite a long time and make the other operation waiting for the global lock fail. + * + * So here after the confirmation of existence of modifying database, we release the global lock + * When dropping database, we will set the `exist` field of db object to false. And in the following + * creation process, we will double-check the `exist` field. + */ + if (getDb(db.getId()) == null) { + throw new DdlException("Database has been dropped when creating table/mv/view"); + } + } finally { + GlobalStateMgr.getCurrentState().unlock(); + } + + if (db.isSystemDatabase()) { + ErrorReport.reportDdlException(ErrorCode.ERR_CANT_CREATE_TABLE, table.getName(), + "cannot create table in system database"); + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + if (!db.isExist()) { + throw new DdlException("Database has been dropped when creating table/mv/view"); + } + + if (!db.registerTableUnlocked(table)) { + if (!isSetIfNotExists) { + table.delete(db.getId(), false); + ErrorReport.reportDdlException(ErrorCode.ERR_CANT_CREATE_TABLE, table.getName(), + "table already exists"); + } else { + LOG.info("Create table[{}] which already exists", table.getName()); + return; + } + } + + // NOTE: The table has been added to the database, and the following procedure cannot throw exception. + LOG.info("Successfully create table: {}-{}, in database: {}-{}", + table.getName(), table.getId(), db.getFullName(), db.getId()); + + CreateTableInfo createTableInfo = new CreateTableInfo(db.getFullName(), table, storageVolumeId); + GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(createTableInfo); + table.onCreate(db); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + @Override + public void createTableLike(CreateTableLikeStmt stmt) throws DdlException { + createTable(stmt.getCreateTableStmt()); + } + + @Override + public void dropTable(DropTableStmt stmt) throws DdlException { + String dbName = stmt.getDbName(); + String tableName = stmt.getTableName(); + + // check database + Database db = getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + if (db.isSystemDatabase()) { + ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, + "cannot drop table in system database: " + db.getOriginName()); + } + db.dropTable(tableName, stmt.isSetIfExists(), stmt.isForceDrop()); + } + + @Override + public void dropTemporaryTable(String dbName, long tableId, String tableName, + boolean isSetIfExsists, boolean isForce) throws DdlException { + Database db = getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + db.dropTemporaryTable(tableId, tableName, isSetIfExsists, isForce); + } + + 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(); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + Table table = getTable(db.getFullName(), tableName); + if (table != null) { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } + + if (!GlobalStateMgr.getCurrentState().getRecycleBin().recoverTable(db, tableName)) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + + Table recoverTable = getTable(db.getFullName(), tableName); + if (recoverTable instanceof OlapTable) { + DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), (OlapTable) recoverTable); + } + + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + /* + * Truncate specified table or partitions. + * The main idea is: + * + * 1. using the same schema to create new table(partitions) + * 2. use the new created table(partitions) to replace the old ones. + * + * if no partition specified, it will truncate all partitions of this table, including all temp partitions, + * otherwise, it will only truncate those specified partitions. + * + */ + @Override + public void truncateTable(TruncateTableStmt truncateTableStmt, ConnectContext context) throws DdlException { + TableRef tblRef = truncateTableStmt.getTblRef(); + TableName dbTbl = tblRef.getName(); + // check, and save some info which need to be checked again later + Map origPartitions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + OlapTable copiedTbl; + Database db = getDb(dbTbl.getDb()); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbTbl.getDb()); + } + + boolean truncateEntireTable = tblRef.getPartitionNames() == null; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + Table table = MetaUtils.getSessionAwareTable(context, db, dbTbl); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTbl.getTbl()); + } + + if (!table.isOlapOrCloudNativeTable()) { + throw new DdlException("Only support truncate OLAP table or LAKE table"); + } + + OlapTable olapTable = (OlapTable) table; + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); + } + + 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); + GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); + } + } else { + for (Partition partition : olapTable.getPartitions()) { + origPartitions.put(partition.getName(), partition); + GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); + } + } + + copiedTbl = getShadowCopyTable(olapTable); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + + // 2. use the copied table to create partitions + List newPartitions = Lists.newArrayListWithCapacity(origPartitions.size()); + // tabletIdSet to save all newly created tablet ids. + Set tabletIdSet = Sets.newHashSet(); + try { + for (Map.Entry entry : origPartitions.entrySet()) { + long oldPartitionId = entry.getValue().getId(); + long newPartitionId = GlobalStateMgr.getCurrentState().getNextId(); + String newPartitionName = entry.getKey(); + + PartitionInfo partitionInfo = copiedTbl.getPartitionInfo(); + partitionInfo.setTabletType(newPartitionId, partitionInfo.getTabletType(oldPartitionId)); + partitionInfo.setIsInMemory(newPartitionId, partitionInfo.getIsInMemory(oldPartitionId)); + partitionInfo.setReplicationNum(newPartitionId, partitionInfo.getReplicationNum(oldPartitionId)); + partitionInfo.setDataProperty(newPartitionId, partitionInfo.getDataProperty(oldPartitionId)); + + if (copiedTbl.isCloudNativeTable()) { + partitionInfo.setDataCacheInfo(newPartitionId, + partitionInfo.getDataCacheInfo(oldPartitionId)); + } + + copiedTbl.setDefaultDistributionInfo(entry.getValue().getDistributionInfo()); + + Partition newPartition = + createPartition(db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, + ConnectContext.get().getCurrentWarehouseId()); + newPartitions.add(newPartition); + } + buildPartitions(db, copiedTbl, newPartitions.stream().map(Partition::getSubPartitions) + .flatMap(p -> p.stream()).collect(Collectors.toList()), ConnectContext.get().getCurrentWarehouseId()); + } catch (DdlException e) { + tabletIdSet.forEach(tabletId -> GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId)); + throw e; + } + Preconditions.checkState(origPartitions.size() == newPartitions.size()); + + // all partitions are created successfully, try to replace the old partitions. + // before replacing, we need to check again. + // Things may be changed outside the database lock. + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) getTable(db.getId(), copiedTbl.getId()); + if (olapTable == null) { + throw new DdlException("Table[" + copiedTbl.getName() + "] is dropped"); + } + + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); + } + + // check partitions + for (Map.Entry entry : origPartitions.entrySet()) { + Partition partition = olapTable.getPartition(entry.getValue().getId()); + if (partition == null || !partition.getName().equalsIgnoreCase(entry.getKey())) { + throw new DdlException("Partition [" + entry.getKey() + "] is changed during truncating table, " + + "please retry"); + } + } + + // check if meta changed + // rollup index may be added or dropped, and schema may be changed during creating partition operation. + boolean metaChanged = false; + if (olapTable.getIndexNameToId().size() != copiedTbl.getIndexNameToId().size()) { + metaChanged = true; + } else { + // compare schemaHash + Map copiedIndexIdToSchemaHash = copiedTbl.getIndexIdToSchemaHash(); + for (Map.Entry entry : olapTable.getIndexIdToSchemaHash().entrySet()) { + long indexId = entry.getKey(); + if (!copiedIndexIdToSchemaHash.containsKey(indexId)) { + metaChanged = true; + break; + } + if (!copiedIndexIdToSchemaHash.get(indexId).equals(entry.getValue())) { + metaChanged = true; + break; + } + } + } + + if (olapTable.getDefaultDistributionInfo().getType() != copiedTbl.getDefaultDistributionInfo().getType()) { + metaChanged = true; + } + + if (metaChanged) { + throw new DdlException("Table[" + copiedTbl.getName() + "]'s meta has been changed. try again."); + } + + // replace + GlobalStateMgr.getCurrentState().getLocalMetastore() + .truncateTableInternal(olapTable, newPartitions, truncateEntireTable, false); + + try { + GlobalStateMgr.getCurrentState().getColocateTableIndex() + .updateLakeTableColocationInfo(olapTable, true /* isJoin */, null /* expectGroupId */); + } catch (DdlException e) { + LOG.info("table {} update colocation info failed when truncate table, {}", olapTable.getId(), e.getMessage()); + } + + // write edit log + TruncateTableInfo info = new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions, + truncateEntireTable); + GlobalStateMgr.getCurrentState().getLocalMetastore().truncateTable(info); + + // refresh mv + Set relatedMvs = olapTable.getRelatedMaterializedViews(); + for (MvId mvId : relatedMvs) { + MaterializedView materializedView = (MaterializedView) getTable(db.getId(), mvId.getId()); + if (materializedView == null) { + LOG.warn("Table related materialized view {} can not be found", mvId.getId()); + continue; + } + if (materializedView.isLoadTriggeredRefresh()) { + refreshMaterializedView(db.getFullName(), getTable(db.getId(), mvId.getId()).getName(), false, null, + Constants.TaskRunPriority.NORMAL.value(), true, false); + } + } + } catch (DdlException e) { + tabletIdSet.forEach(tabletId -> GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId)); + throw e; + } catch (MetaNotFoundException e) { + LOG.warn("Table related materialized view can not be found", e); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + + LOG.info("finished to truncate table {}, partitions: {}", + tblRef.getName().toSql(), tblRef.getPartitionNames()); + } + + /* + * used for handling AlterTableStmt (for client is the ALTER TABLE command). + * including SchemaChangeHandler and RollupHandler + */ + @Override + public void alterTable(ConnectContext context, AlterTableStmt stmt) throws UserException { + AlterJobExecutor alterJobExecutor = new AlterJobExecutor(); + alterJobExecutor.process(stmt, context); + } + + @Override + public void alterTableComment(Database db, Table table, AlterTableCommentClause clause) { + ModifyTablePropertyOperationLog log = new ModifyTablePropertyOperationLog(db.getId(), table.getId()); + log.setComment(clause.getNewComment()); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(log); + table.setComment(clause.getNewComment()); + } + + @Override + public void renameTable(Database db, Table table, TableRenameClause tableRenameClause) throws DdlException { + OlapTable olapTable = (OlapTable) table; + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); + } + + String oldTableName = olapTable.getName(); + String newTableName = tableRenameClause.getNewTableName(); + if (oldTableName.equals(newTableName)) { + throw new DdlException("Same table name"); + } + + // check if name is already used + if (getTable(db.getFullName(), newTableName) != null) { + throw new DdlException("Table name[" + newTableName + "] is already used"); + } + + olapTable.checkAndSetName(newTableName, false); + + db.dropTable(oldTableName); + db.registerTableUnlocked(olapTable); + inactiveRelatedMaterializedView(db, olapTable, + MaterializedViewExceptions.inactiveReasonForBaseTableRenamed(oldTableName)); + + TableInfo tableInfo = TableInfo.createForTableRename(db.getId(), olapTable.getId(), newTableName); + GlobalStateMgr.getCurrentState().getLocalMetastore().renameTable(tableInfo); + LOG.info("rename table[{}] to {}, tableId: {}", oldTableName, newTableName, olapTable.getId()); + } + + @Override + public Table getTable(String dbName, String tblName) { + return GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(dbName, tblName); + } + + public Table getTable(Long dbId, Long tableId) { + return GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(dbId, tableId); + } + + public Optional
mayGetTable(long dbId, long tableId) { + return mayGetDb(dbId).flatMap(db -> Optional.ofNullable(db.getTable(tableId))); + } + + public Optional
mayGetTable(String dbName, String tableName) { + return mayGetDb(dbName).flatMap(db -> Optional.ofNullable(db.getTable(tableName))); + } + + @Override + public void createView(CreateViewStmt stmt) throws DdlException { + String dbName = stmt.getDbName(); + 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); + } + // check if table exists in db + boolean existed = false; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + if (getTable(db.getFullName(), tableName) != null) { + existed = true; + if (stmt.isSetIfNotExists()) { + LOG.info("create view[{}] which already exists", tableName); + return; + } else if (stmt.isReplace()) { + LOG.info("view {} already exists, need to replace it", tableName); + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); + } + } + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + + if (existed) { + // already existed, need to alter the view + AlterViewStmt alterViewStmt = AlterViewStmt.fromReplaceStmt(stmt); + new AlterJobExecutor().process(alterViewStmt, ConnectContext.get()); + LOG.info("replace view {} successfully", tableName); + } else { + List columns = stmt.getColumns(); + long tableId = GlobalStateMgr.getCurrentState().getNextId(); + View view = new View(tableId, tableName, columns); + view.setComment(stmt.getComment()); + view.setInlineViewDefWithSqlMode(stmt.getInlineViewDef(), + ConnectContext.get().getSessionVariable().getSqlMode()); + // init here in case the stmt string from view.toSql() has some syntax error. + try { + view.init(); + } catch (UserException e) { + throw new DdlException("failed to init view stmt", e); + } + + onCreate(db, view, "", stmt.isSetIfNotExists()); + LOG.info("successfully create view[" + tableName + "-" + view.getId() + "]"); + } + } + + /** + * used for handling AlterViewStmt (the ALTER VIEW command). + */ + @Override + public void alterView(AlterViewStmt stmt) { + new AlterJobExecutor().process(stmt, ConnectContext.get()); + } + + @Override + public void createMaterializedView(CreateMaterializedViewStmt stmt) + throws AnalysisException, DdlException { + MaterializedViewHandler materializedViewHandler = + GlobalStateMgr.getCurrentState().getAlterJobMgr().getMaterializedViewHandler(); + String tableName = stmt.getBaseIndexName(); + // check db + String dbName = stmt.getDBName(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + // check cluster capacity + GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().checkClusterCapacity(); + // check db quota + db.checkQuota(); + + Locker locker = new Locker(); + if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { + throw new DdlException("create materialized failed. database:" + db.getFullName() + " not exist"); + } + try { + Table table = getTable(db.getFullName(), tableName); + if (table == null) { + throw new DdlException("create materialized failed. table:" + tableName + " not exist"); + } + if (table.isCloudNativeTable()) { + throw new DdlException("Creating synchronous materialized view(rollup) is not supported in " + + "shared data clusters.\nPlease use asynchronous materialized view instead.\n" + + "Refer to https://docs.starrocks.io/en-us/latest/sql-reference/sql-statements" + + "/data-definition/CREATE%20MATERIALIZED%20VIEW#asynchronous-materialized-view for details."); + } + if (!table.isOlapTable()) { + throw new DdlException("Do not support create synchronous materialized view(rollup) on " + + table.getType().name() + " table[" + tableName + "]"); + } + OlapTable olapTable = (OlapTable) table; + if (olapTable.getKeysType() == KeysType.PRIMARY_KEYS) { + throw new DdlException( + "Do not support create materialized view on primary key table[" + tableName + "]"); + } + if (GlobalStateMgr.getCurrentState().getInsertOverwriteJobMgr().hasRunningOverwriteJob(olapTable.getId())) { + throw new DdlException("Table[" + olapTable.getName() + "] is doing insert overwrite job, " + + "please start to create materialized view after insert overwrite"); + } + olapTable.checkStableAndNormal(); + + materializedViewHandler.processCreateMaterializedView(stmt, db, olapTable); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + // TODO(murphy) refactor it into MVManager + @Override + public void createMaterializedView(CreateMaterializedViewStatement stmt) + throws DdlException { + // check mv exists,name must be different from view/mv/table which exists in metadata + String mvName = stmt.getTableName().getTbl(); + String dbName = stmt.getTableName().getDb(); + LOG.debug("Begin create materialized view: {}", mvName); + // check if db exists + Database db = this.getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + + // check if table exists in db + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + if (getTable(db.getFullName(), mvName) != null) { + if (stmt.isIfNotExists()) { + LOG.info("Create materialized view [{}] which already exists", mvName); + return; + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, mvName); + } + } + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + // create columns + List baseSchema = stmt.getMvColumnItems(); + validateColumns(baseSchema); + + Map properties = stmt.getProperties(); + if (properties == null) { + properties = Maps.newHashMap(); + } + + // create partition info + PartitionInfo partitionInfo = buildPartitionInfo(stmt); + // create distribution info + DistributionDesc distributionDesc = stmt.getDistributionDesc(); + Preconditions.checkNotNull(distributionDesc); + DistributionInfo baseDistribution = distributionDesc.toDistributionInfo(baseSchema); + // create refresh scheme + MaterializedView.MvRefreshScheme mvRefreshScheme; + RefreshSchemeClause refreshSchemeDesc = stmt.getRefreshSchemeDesc(); + if (refreshSchemeDesc.getType() == MaterializedView.RefreshType.ASYNC) { + mvRefreshScheme = new MaterializedView.MvRefreshScheme(); + AsyncRefreshSchemeDesc asyncRefreshSchemeDesc = (AsyncRefreshSchemeDesc) refreshSchemeDesc; + MaterializedView.AsyncRefreshContext asyncRefreshContext = mvRefreshScheme.getAsyncRefreshContext(); + asyncRefreshContext.setDefineStartTime(asyncRefreshSchemeDesc.isDefineStartTime()); + int randomizeStart = 0; + if (properties.containsKey(PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START)) { + try { + randomizeStart = Integer.parseInt(properties.get((PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START))); + } catch (NumberFormatException e) { + ErrorReport.reportSemanticException(ErrorCode.ERR_INVALID_PARAMETER, + PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START + " only accept integer as parameter"); + } + // remove this transient variable + properties.remove(PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START); + } + + long random = getRandomStart(asyncRefreshSchemeDesc.getIntervalLiteral(), randomizeStart); + if (asyncRefreshSchemeDesc.isDefineStartTime() || randomizeStart == -1) { + long definedStartTime = Utils.getLongFromDateTime(asyncRefreshSchemeDesc.getStartTime()); + // Add random set only if mv_random_start > 0 when user has already set the start time + if (randomizeStart > 0) { + definedStartTime += random; + } + asyncRefreshContext.setStartTime(definedStartTime); + } else if (asyncRefreshSchemeDesc.getIntervalLiteral() != null) { + long currentTimeSecond = Utils.getLongFromDateTime(LocalDateTime.now()); + long randomizedStart = currentTimeSecond + random; + asyncRefreshContext.setStartTime(randomizedStart); + } + if (asyncRefreshSchemeDesc.getIntervalLiteral() != null) { + long intervalStep = ((IntLiteral) asyncRefreshSchemeDesc.getIntervalLiteral().getValue()).getValue(); + String refreshTimeUnit = asyncRefreshSchemeDesc.getIntervalLiteral().getUnitIdentifier().getDescription(); + asyncRefreshContext.setStep(intervalStep); + asyncRefreshContext.setTimeUnit(refreshTimeUnit); + + // Check the interval time should not be less than the min allowed config time. + if (Config.materialized_view_min_refresh_interval > 0) { + TimeUnit intervalTimeUnit = TimeUtils.convertUnitIdentifierToTimeUnit(refreshTimeUnit); + long periodSeconds = TimeUtils.convertTimeUnitValueToSecond(intervalStep, intervalTimeUnit); + if (periodSeconds < Config.materialized_view_min_refresh_interval) { + throw new DdlException(String.format("Refresh schedule interval %s is too small which may cost " + + "a lot of memory/cpu resources to refresh the asynchronous materialized view, " + + "please config an interval larger than " + + "Config.materialized_view_min_refresh_interval(%ss).", + periodSeconds, + Config.materialized_view_min_refresh_interval)); + } + } + } + + // task which type is EVENT_TRIGGERED can not use external table as base table now. + if (asyncRefreshContext.getTimeUnit() == null) { + // asyncRefreshContext's timeUnit is null means this task's type is EVENT_TRIGGERED + Map tableNameTableMap = AnalyzerUtils.collectAllTable(stmt.getQueryStatement()); + if (tableNameTableMap.values().stream().anyMatch(table -> !table.isNativeTableOrMaterializedView())) { + throw new DdlException( + "Materialized view which type is ASYNC need to specify refresh interval for " + + "external table"); + } + } + } else if (refreshSchemeDesc.getType() == MaterializedView.RefreshType.SYNC) { + mvRefreshScheme = new MaterializedView.MvRefreshScheme(); + mvRefreshScheme.setType(MaterializedView.RefreshType.SYNC); + } else if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.MANUAL)) { + mvRefreshScheme = new MaterializedView.MvRefreshScheme(); + mvRefreshScheme.setType(MaterializedView.RefreshType.MANUAL); + } else { + mvRefreshScheme = new MaterializedView.MvRefreshScheme(); + mvRefreshScheme.setType(MaterializedView.RefreshType.INCREMENTAL); + } + mvRefreshScheme.setMoment(refreshSchemeDesc.getMoment()); + // create mv + long mvId = GlobalStateMgr.getCurrentState().getNextId(); + MaterializedView materializedView; + if (RunMode.isSharedNothingMode()) { + if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.INCREMENTAL)) { + materializedView = GlobalStateMgr.getCurrentState().getMaterializedViewMgr() + .createSinkTable(stmt, partitionInfo, mvId, db.getId()); + materializedView.setMaintenancePlan(stmt.getMaintenancePlan()); + } else { + materializedView = + new MaterializedView(mvId, db.getId(), mvName, baseSchema, stmt.getKeysType(), partitionInfo, + baseDistribution, mvRefreshScheme); + } + } else { + Preconditions.checkState(RunMode.isSharedDataMode()); + if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.INCREMENTAL)) { + throw new DdlException("Incremental materialized view in shared_data mode is not supported"); + } + + materializedView = + new LakeMaterializedView(mvId, db.getId(), mvName, baseSchema, stmt.getKeysType(), partitionInfo, + baseDistribution, mvRefreshScheme); + } + + //bitmap indexes + List mvIndexes = stmt.getMvIndexes(); + materializedView.setIndexes(mvIndexes); + + // sort keys + if (CollectionUtils.isNotEmpty(stmt.getSortKeys())) { + materializedView.setTableProperty(new TableProperty()); + materializedView.getTableProperty().setMvSortKeys(stmt.getSortKeys()); + } + // set comment + materializedView.setComment(stmt.getComment()); + // set baseTableIds + materializedView.setBaseTableInfos(stmt.getBaseTableInfos()); + // set viewDefineSql + materializedView.setViewDefineSql(stmt.getInlineViewDef()); + materializedView.setSimpleDefineSql(stmt.getSimpleViewDef()); + materializedView.setOriginalViewDefineSql(stmt.getOriginalViewDefineSql()); + // set partitionRefTableExprs + if (stmt.getPartitionRefTableExpr() != null) { + //avoid to get a list of null inside + materializedView.setPartitionRefTableExprs(Lists.newArrayList(stmt.getPartitionRefTableExpr())); + } + // set base index id + long baseIndexId = GlobalStateMgr.getCurrentState().getNextId(); + materializedView.setBaseIndexId(baseIndexId); + // set query output indexes + materializedView.setQueryOutputIndices(stmt.getQueryOutputIndices()); + // set base index meta + int schemaVersion = 0; + int schemaHash = Util.schemaHash(schemaVersion, baseSchema, null, 0d); + short shortKeyColumnCount = GlobalStateMgr.calcShortKeyColumnCount(baseSchema, null); + TStorageType baseIndexStorageType = TStorageType.COLUMN; + materializedView.setIndexMeta(baseIndexId, mvName, baseSchema, schemaVersion, schemaHash, + shortKeyColumnCount, baseIndexStorageType, stmt.getKeysType()); + + // validate hint + Map optHints = Maps.newHashMap(); + if (stmt.isExistQueryScopeHint()) { + SessionVariable sessionVariable = VariableMgr.newSessionVariable(); + for (HintNode hintNode : stmt.getAllQueryScopeHints()) { + if (hintNode instanceof SetVarHint) { + for (Map.Entry entry : hintNode.getValue().entrySet()) { + VariableMgr.setSystemVariable(sessionVariable, + new SystemVariable(entry.getKey(), new StringLiteral(entry.getValue())), true); + optHints.put(entry.getKey(), entry.getValue()); + } + } else if (hintNode instanceof UserVariableHint) { + throw new DdlException("unsupported user variable hint in Materialized view for now."); + } + } + } + + boolean isNonPartitioned = partitionInfo.isUnPartitioned(); + DataProperty dataProperty = PropertyAnalyzer.analyzeMVDataProperty(materializedView, properties); + PropertyAnalyzer.analyzeMVProperties(db, materializedView, properties, isNonPartitioned); + try { + Set tabletIdSet = new HashSet<>(); + // process single partition info + if (isNonPartitioned) { + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); + Preconditions.checkNotNull(dataProperty); + partitionInfo.setDataProperty(partitionId, dataProperty); + partitionInfo.setReplicationNum(partitionId, materializedView.getDefaultReplicationNum()); + partitionInfo.setIsInMemory(partitionId, false); + partitionInfo.setTabletType(partitionId, TTabletType.TABLET_TYPE_DISK); + StorageInfo storageInfo = materializedView.getTableProperty().getStorageInfo(); + partitionInfo.setDataCacheInfo(partitionId, + storageInfo == null ? null : storageInfo.getDataCacheInfo()); + Long version = Partition.PARTITION_INIT_VERSION; + Partition partition = createPartition(db, materializedView, partitionId, mvName, version, tabletIdSet, + materializedView.getWarehouseId()); + buildPartitions(db, materializedView, new ArrayList<>(partition.getSubPartitions()), + materializedView.getWarehouseId()); + materializedView.addPartition(partition); + } else { + Expr partitionExpr = stmt.getPartitionExpDesc().getExpr(); + Map partitionExprMaps = MVPartitionExprResolver.getMVPartitionExprsChecked(partitionExpr, + stmt.getQueryStatement(), stmt.getBaseTableInfos()); + LOG.info("Generate mv {} partition exprs: {}", mvName, partitionExprMaps); + materializedView.setPartitionExprMaps(partitionExprMaps); + } + + GlobalStateMgr.getCurrentState().getMaterializedViewMgr().prepareMaintenanceWork(stmt, materializedView); + + String storageVolumeId = ""; + if (materializedView.isCloudNativeMaterializedView()) { + storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr() + .getStorageVolumeIdOfTable(materializedView.getId()); + } + onCreate(db, materializedView, storageVolumeId, stmt.isIfNotExists()); + } catch (DdlException e) { + if (materializedView.isCloudNativeMaterializedView()) { + GlobalStateMgr.getCurrentState().getStorageVolumeMgr().unbindTableToStorageVolume(materializedView.getId()); + } + throw e; + } + LOG.info("Successfully create materialized view [{}:{}]", mvName, materializedView.getMvId()); + + // NOTE: The materialized view has been added to the database, and the following procedure cannot throw exception. + createTaskForMaterializedView(dbName, materializedView, optHints); + DynamicPartitionUtil.registerOrRemovePartitionTTLTable(db.getId(), materializedView); + } + + private long getRandomStart(IntervalLiteral interval, long randomizeStart) throws DdlException { + if (interval == null || randomizeStart == -1) { + return 0; + } + // randomize the start time if not specified manually, to avoid refresh conflicts + // default random interval is min(300s, INTERVAL/2) + // user could specify it through mv_randomize_start + long period = ((IntLiteral) interval.getValue()).getLongValue(); + TimeUnit timeUnit = + TimeUtils.convertUnitIdentifierToTimeUnit(interval.getUnitIdentifier().getDescription()); + long intervalSeconds = TimeUtils.convertTimeUnitValueToSecond(period, timeUnit); + long randomInterval = randomizeStart == 0 ? Math.min(300, intervalSeconds / 2) : randomizeStart; + return randomInterval > 0 ? ThreadLocalRandom.current().nextLong(randomInterval) : randomInterval; + } + + private void createTaskForMaterializedView(String dbName, MaterializedView materializedView, + Map optHints) throws DdlException { + MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); + MaterializedView.RefreshMoment refreshMoment = materializedView.getRefreshScheme().getMoment(); + + if (refreshType.equals(MaterializedView.RefreshType.INCREMENTAL)) { + GlobalStateMgr.getCurrentState().getMaterializedViewMgr().startMaintainMV(materializedView); + return; + } + + if (refreshType != MaterializedView.RefreshType.SYNC) { + + Task task = TaskBuilder.buildMvTask(materializedView, dbName); + TaskBuilder.updateTaskInfo(task, materializedView); + + if (optHints != null) { + Map taskProperties = task.getProperties(); + taskProperties.putAll(optHints); + } + + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + taskManager.createTask(task, false); + if (refreshMoment.equals(MaterializedView.RefreshMoment.IMMEDIATE)) { + taskManager.executeTask(task.getName()); + } + } + } + + /** + * Leave some clean up work to {@link MaterializedView#onDrop} + */ + @Override + public void dropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { + Database db = getDb(stmt.getDbName()); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDbName()); + } + Table table; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + table = getTable(db.getFullName(), stmt.getMvName()); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + if (table instanceof MaterializedView) { + try { + Authorizer.checkMaterializedViewAction(ConnectContext.get().getCurrentUserIdentity(), + ConnectContext.get().getCurrentRoleIds(), stmt.getDbMvName(), PrivilegeType.DROP); + } catch (AccessDeniedException e) { + AccessDeniedException.reportAccessDenied( + stmt.getDbMvName().getCatalog(), + ConnectContext.get().getCurrentUserIdentity(), + ConnectContext.get().getCurrentRoleIds(), PrivilegeType.DROP.name(), ObjectType.MATERIALIZED_VIEW.name(), + stmt.getDbMvName().getTbl()); + } + + db.dropTable(table.getName(), stmt.isSetIfExists(), true); + } else { + GlobalStateMgr.getCurrentState().getAlterJobMgr().processDropMaterializedView(stmt); + } + } + + @Override + public String refreshMaterializedView(RefreshMaterializedViewStatement refreshMaterializedViewStatement) + throws DdlException, MetaNotFoundException { + String dbName = refreshMaterializedViewStatement.getMvName().getDb(); + String mvName = refreshMaterializedViewStatement.getMvName().getTbl(); + boolean force = refreshMaterializedViewStatement.isForceRefresh(); + PartitionRangeDesc range = refreshMaterializedViewStatement.getPartitionRangeDesc(); + return refreshMaterializedView(dbName, mvName, force, range, Constants.TaskRunPriority.HIGH.value(), + Config.enable_mv_refresh_sync_refresh_mergeable, true, refreshMaterializedViewStatement.isSync()); + } + + @Override + public void cancelRefreshMaterializedView( + CancelRefreshMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { + String dbName = stmt.getMvName().getDb(); + String mvName = stmt.getMvName().getTbl(); + MaterializedView materializedView = getMaterializedViewToRefresh(dbName, mvName); + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + Task refreshTask = taskManager.getTask(TaskBuilder.getMvTaskName(materializedView.getId())); + boolean isForce = stmt.isForce(); + if (refreshTask != null) { + taskManager.killTask(refreshTask.getName(), isForce); + } + } + + private String executeRefreshMvTask(String dbName, MaterializedView materializedView, + ExecuteOption executeOption) + throws DdlException { + MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); + LOG.info("Start to execute refresh materialized view task, mv: {}, refreshType: {}, executionOption:{}", + materializedView.getName(), refreshType, executeOption); + + if (refreshType.equals(MaterializedView.RefreshType.INCREMENTAL)) { + GlobalStateMgr.getCurrentState().getMaterializedViewMgr().onTxnPublish(materializedView); + } else if (refreshType != MaterializedView.RefreshType.SYNC) { + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + final String mvTaskName = TaskBuilder.getMvTaskName(materializedView.getId()); + if (!taskManager.containTask(mvTaskName)) { + Task task = TaskBuilder.buildMvTask(materializedView, dbName); + TaskBuilder.updateTaskInfo(task, materializedView); + taskManager.createTask(task, false); + } + return taskManager.executeTask(mvTaskName, executeOption).getQueryId(); + } + return null; + } + + private MaterializedView getMaterializedViewToRefresh(String dbName, String mvName) + throws DdlException, MetaNotFoundException { + Database db = this.getDb(dbName); + if (db == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); + } + final Table table = getTable(db.getFullName(), mvName); + MaterializedView materializedView = null; + if (table instanceof MaterializedView) { + materializedView = (MaterializedView) table; + } + if (materializedView == null) { + throw new MetaNotFoundException(mvName + " is not a materialized view"); + } + return materializedView; + } + + public String refreshMaterializedView(String dbName, String mvName, boolean force, PartitionRangeDesc range, + int priority, boolean mergeRedundant, boolean isManual) + throws DdlException, MetaNotFoundException { + return refreshMaterializedView(dbName, mvName, force, range, priority, mergeRedundant, isManual, false); + } + + public String refreshMaterializedView(String dbName, String mvName, boolean force, PartitionRangeDesc range, + int priority, boolean mergeRedundant, boolean isManual, boolean isSync) + throws DdlException, MetaNotFoundException { + MaterializedView materializedView = getMaterializedViewToRefresh(dbName, mvName); + + HashMap taskRunProperties = new HashMap<>(); + taskRunProperties.put(TaskRun.PARTITION_START, range == null ? null : range.getPartitionStart()); + taskRunProperties.put(TaskRun.PARTITION_END, range == null ? null : range.getPartitionEnd()); + taskRunProperties.put(TaskRun.FORCE, Boolean.toString(force)); + + ExecuteOption executeOption = new ExecuteOption(priority, mergeRedundant, taskRunProperties); + executeOption.setManual(isManual); + executeOption.setSync(isSync); + return executeRefreshMvTask(dbName, materializedView, executeOption); + } + + @Override + public void alterMaterializedView(AlterMaterializedViewStmt stmt) { + new AlterMVJobExecutor().process(stmt, ConnectContext.get()); + } + + @Override + public void addPartitions(ConnectContext ctx, Database db, String tableName, AddPartitionClause addPartitionClause) + throws DdlException { + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + Table table = getTable(db.getFullName(), tableName); + CatalogUtils.checkTableExist(db, tableName); + CatalogUtils.checkNativeTable(db, table); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + addPartitions(ctx, db, tableName, + addPartitionClause.getResolvedPartitionDescList(), + addPartitionClause.isTempPartition(), + addPartitionClause.getDistributionDesc()); + } + + private void addPartitions(ConnectContext ctx, Database db, String tableName, List partitionDescs, + boolean isTempPartition, DistributionDesc distributionDesc) throws DdlException { + DistributionInfo distributionInfo; + OlapTable olapTable; + OlapTable copiedTable; + + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + Set checkExistPartitionName = Sets.newConcurrentHashSet(); + try { + olapTable = checkTable(db, tableName); + + // get partition info + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + + // check partition type + checkPartitionType(partitionInfo); + + // check partition num + checkPartitionNum(olapTable); + + // get distributionInfo + distributionInfo = getDistributionInfo(olapTable, distributionDesc).copy(); + olapTable.inferDistribution(distributionInfo); + + // check colocation + checkColocation(db, olapTable, distributionInfo, partitionDescs); + copiedTable = getShadowCopyTable(olapTable); + copiedTable.setDefaultDistributionInfo(distributionInfo); + checkExistPartitionName = CatalogUtils.checkPartitionNameExistForAddPartitions(olapTable, partitionDescs); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + + Preconditions.checkNotNull(distributionInfo); + Preconditions.checkNotNull(olapTable); + Preconditions.checkNotNull(copiedTable); + + // create partition outside db lock + checkDataProperty(partitionDescs); + + Set tabletIdSetForAll = Sets.newHashSet(); + HashMap> partitionNameToTabletSet = Maps.newHashMap(); + try { + // create partition list + List> newPartitions = + createPartitionMap(db, copiedTable, partitionDescs, partitionNameToTabletSet, tabletIdSetForAll, + checkExistPartitionName, ctx.getCurrentWarehouseId()); + + // build partitions + List partitionList = newPartitions.stream().map(x -> x.first).collect(Collectors.toList()); + buildPartitions(db, copiedTable, partitionList.stream().map(Partition::getSubPartitions) + .flatMap(p -> p.stream()).collect(Collectors.toList()), ctx.getCurrentWarehouseId()); + + // check again + if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { + throw new DdlException("db " + db.getFullName() + + "(" + db.getId() + ") has been dropped"); + } + Set existPartitionNameSet = Sets.newHashSet(); + try { + olapTable = checkTable(db, tableName); + existPartitionNameSet = CatalogUtils.checkPartitionNameExistForAddPartitions(olapTable, + partitionDescs); + if (existPartitionNameSet.size() > 0) { + for (String partitionName : existPartitionNameSet) { + LOG.info("add partition[{}] which already exists", partitionName); + } + } + + // check if meta changed + checkIfMetaChange(olapTable, copiedTable, tableName); + + // get partition info + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + + // check partition type + checkPartitionType(partitionInfo); + + // update partition info + updatePartitionInfo(partitionInfo, newPartitions, existPartitionNameSet, isTempPartition, olapTable); + + try { + GlobalStateMgr.getCurrentState().getColocateTableIndex() + .updateLakeTableColocationInfo(olapTable, true /* isJoin */, null /* expectGroupId */); + } catch (DdlException e) { + LOG.info("table {} update colocation info failed when add partition, {}", olapTable.getId(), e.getMessage()); + } + + // add partition log + GlobalStateMgr.getCurrentState().getLocalMetastore(). + addPartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, + existPartitionNameSet); + } finally { + cleanExistPartitionNameSet(existPartitionNameSet, partitionNameToTabletSet); + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } catch (DdlException e) { + cleanTabletIdSetForAll(tabletIdSetForAll); + throw e; + } + } + + private void cleanTabletIdSetForAll(Set tabletIdSetForAll) { + // Cleanup of shards for LakeTable is taken care by ShardDeleter + for (Long tabletId : tabletIdSetForAll) { + GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); + } + } + + private void cleanExistPartitionNameSet(Set existPartitionNameSet, + HashMap> partitionNameToTabletSet) { + for (String partitionName : existPartitionNameSet) { + Set existPartitionTabletSet = partitionNameToTabletSet.get(partitionName); + if (existPartitionTabletSet == null) { + // should not happen + continue; + } + for (Long tabletId : existPartitionTabletSet) { + // createPartitionWithIndices create duplicate tablet that if not exists scenario + // so here need to clean up those created tablets which partition already exists from invert index + GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); + } + } + } + + private void checkPartitionNum(OlapTable olapTable) throws DdlException { + if (olapTable.getNumberOfPartitions() > Config.max_partition_number_per_table) { + throw new DdlException("Table " + olapTable.getName() + " created partitions exceeded the maximum limit: " + + Config.max_partition_number_per_table + ". You can modify this restriction on by setting" + + " max_partition_number_per_table larger."); + } + } + + @Override + public void dropPartition(Database db, Table table, DropPartitionClause clause) throws DdlException { + CatalogUtils.checkTableExist(db, table.getName()); + Locker locker = new Locker(); + OlapTable olapTable = (OlapTable) table; + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); + } + if (!partitionInfo.isRangePartition() && partitionInfo.getType() != PartitionType.LIST) { + throw new DdlException("Alter table [" + olapTable.getName() + "] failed. Not a partitioned table"); + } + boolean isTempPartition = clause.isTempPartition(); + + List existPartitions = Lists.newArrayList(); + List notExistPartitions = Lists.newArrayList(); + for (String partitionName : clause.getResolvedPartitionNames()) { + if (olapTable.checkPartitionNameExist(partitionName, isTempPartition)) { + existPartitions.add(partitionName); + } else { + notExistPartitions.add(partitionName); + } + } + if (CollectionUtils.isNotEmpty(notExistPartitions)) { + if (clause.isSetIfExists()) { + LOG.info("drop partition[{}] which does not exist", notExistPartitions); + } else { + ErrorReport.reportDdlException(ErrorCode.ERR_DROP_PARTITION_NON_EXISTENT, notExistPartitions); + } + } + if (CollectionUtils.isEmpty(existPartitions)) { + return; + } + for (String partitionName : existPartitions) { + // drop + if (isTempPartition) { + olapTable.dropTempPartition(partitionName, true); + } else { + Partition partition = olapTable.getPartition(partitionName); + if (!clause.isForceDrop()) { + if (partition != null) { + if (GlobalStateMgr.getCurrentState().getGlobalTransactionMgr() + .existCommittedTxns(db.getId(), olapTable.getId(), partition.getId())) { + throw new DdlException( + "There are still some transactions in the COMMITTED state waiting to be completed." + + " The partition [" + partitionName + + "] cannot be dropped. If you want to forcibly drop(cannot be recovered)," + + " please use \"DROP PARTITION FORCE\"."); + } + } + } + Range partitionRange = null; + if (partition != null) { + GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); + if (partitionInfo instanceof RangePartitionInfo) { + partitionRange = ((RangePartitionInfo) partitionInfo).getRange(partition.getId()); + } + } + + olapTable.dropPartition(db.getId(), partitionName, clause.isForceDrop()); + if (olapTable instanceof MaterializedView) { + MaterializedView mv = (MaterializedView) olapTable; + SyncPartitionUtils.dropBaseVersionMeta(mv, partitionName, partitionRange); + } + } + } + if (!isTempPartition) { + try { + for (MvId mvId : olapTable.getRelatedMaterializedViews()) { + MaterializedView materializedView = (MaterializedView) getTable(db.getId(), mvId.getId()); + if (materializedView != null && materializedView.isLoadTriggeredRefresh()) { + refreshMaterializedView( + db.getFullName(), materializedView.getName(), false, null, + Constants.TaskRunPriority.NORMAL.value(), true, false); + } + } + } catch (MetaNotFoundException e) { + throw new DdlException("fail to refresh materialized views when dropping partition", e); + } + } + long dbId = db.getId(); + long tableId = olapTable.getId(); + + + if (clause.getPartitionName() != null) { + String partitionName = clause.getPartitionName(); + DropPartitionInfo info = new DropPartitionInfo(dbId, tableId, partitionName, isTempPartition, clause.isForceDrop()); + GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(info); + LOG.info("succeed in dropping partition[{}], is temp : {}, is force : {}", partitionName, isTempPartition, + clause.isForceDrop()); + } else { + DropPartitionsInfo info = + new DropPartitionsInfo(dbId, tableId, isTempPartition, clause.isForceDrop(), existPartitions); + GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartitions(info); + LOG.info("succeed in dropping partitions[{}], is temp : {}, is force : {}", existPartitions, isTempPartition, + clause.isForceDrop()); + } + } + + 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(); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + Table table = getTable(db.getFullName(), tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + + if (!table.isOlapOrCloudNativeTable()) { + throw new DdlException("table[" + tableName + "] is not OLAP table or LAKE table"); + } + OlapTable olapTable = (OlapTable) table; + + String partitionName = recoverStmt.getPartitionName(); + if (olapTable.getPartition(partitionName) != null) { + throw new DdlException("partition[" + partitionName + "] already exist in table[" + tableName + "]"); + } + + GlobalStateMgr.getCurrentState().getRecycleBin().recoverPartition(db.getId(), olapTable, partitionName); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + private OlapTable checkTable(Database db, String tableName) throws DdlException { + CatalogUtils.checkTableExist(db, tableName); + Table table = getTable(db.getFullName(), tableName); + CatalogUtils.checkNativeTable(db, table); + OlapTable olapTable = (OlapTable) table; + CatalogUtils.checkTableState(olapTable, tableName); + return olapTable; + } + + private OlapTable checkTable(Database db, Long tableId) throws DdlException { + Table table = getTable(db.getId(), tableId); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableId); + } + CatalogUtils.checkNativeTable(db, table); + OlapTable olapTable = (OlapTable) table; + CatalogUtils.checkTableState(olapTable, table.getName()); + return olapTable; + } + + private void checkPartitionType(PartitionInfo partitionInfo) throws DdlException { + PartitionType partitionType = partitionInfo.getType(); + if (!partitionInfo.isRangePartition() && partitionType != PartitionType.LIST) { + throw new DdlException("Only support adding partition to range/list partitioned table"); + } + } + + private DistributionInfo getDistributionInfo(OlapTable olapTable, DistributionDesc distributionDesc) + throws DdlException { + DistributionInfo distributionInfo; + List baseSchema = olapTable.getBaseSchema(); + DistributionInfo defaultDistributionInfo = olapTable.getDefaultDistributionInfo(); + if (distributionDesc != null) { + distributionInfo = distributionDesc.toDistributionInfo(baseSchema); + // for now. we only support modify distribution's bucket num + if (distributionInfo.getType() != defaultDistributionInfo.getType()) { + throw new DdlException("Cannot assign different distribution type. default is: " + + defaultDistributionInfo.getType()); + } + + if (distributionInfo.getType() == DistributionInfo.DistributionInfoType.HASH) { + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; + List newDistriCols = MetaUtils.getColumnsByColumnIds(olapTable, + hashDistributionInfo.getDistributionColumns()); + List defaultDistriCols = MetaUtils.getColumnsByColumnIds(olapTable, + defaultDistributionInfo.getDistributionColumns()); + if (!newDistriCols.equals(defaultDistriCols)) { + throw new DdlException("Cannot assign hash distribution with different distribution cols. " + + "default is: " + defaultDistriCols); + } + if (hashDistributionInfo.getBucketNum() < 0) { + throw new DdlException("Cannot assign hash distribution buckets less than 0"); + } + } + if (distributionInfo.getType() == DistributionInfo.DistributionInfoType.RANDOM) { + RandomDistributionInfo randomDistributionInfo = (RandomDistributionInfo) distributionInfo; + if (randomDistributionInfo.getBucketNum() < 0) { + throw new DdlException("Cannot assign random distribution buckets less than 0"); + } + } + } else { + distributionInfo = defaultDistributionInfo; + } + return distributionInfo; + } + + private void checkColocation(Database db, OlapTable olapTable, DistributionInfo distributionInfo, + List partitionDescs) + throws DdlException { + if (GlobalStateMgr.getCurrentState().getColocateTableIndex().isColocateTable(olapTable.getId())) { + String fullGroupName = db.getId() + "_" + olapTable.getColocateGroup(); + ColocateGroupSchema groupSchema = GlobalStateMgr.getCurrentState() + .getColocateTableIndex().getGroupSchema(fullGroupName); + Preconditions.checkNotNull(groupSchema); + groupSchema.checkDistribution(olapTable.getIdToColumn(), distributionInfo); + for (PartitionDesc partitionDesc : partitionDescs) { + groupSchema.checkReplicationNum(partitionDesc.getReplicationNum()); + } + } + } + + private void checkDataProperty(List partitionDescs) { + for (PartitionDesc partitionDesc : partitionDescs) { + DataProperty dataProperty = partitionDesc.getPartitionDataProperty(); + Preconditions.checkNotNull(dataProperty); + } + } + + private List> createPartitionMap(Database db, OlapTable copiedTable, + List partitionDescs, + HashMap> partitionNameToTabletSet, + Set tabletIdSetForAll, + Set existPartitionNameSet, + long warehouseId) + throws DdlException { + List> partitionList = Lists.newArrayList(); + for (PartitionDesc partitionDesc : partitionDescs) { + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); + DataProperty dataProperty = partitionDesc.getPartitionDataProperty(); + String partitionName = partitionDesc.getPartitionName(); + if (existPartitionNameSet.contains(partitionName)) { + continue; + } + Long version = partitionDesc.getVersionInfo(); + Set tabletIdSet = Sets.newHashSet(); + + copiedTable.getPartitionInfo().setDataProperty(partitionId, dataProperty); + copiedTable.getPartitionInfo().setTabletType(partitionId, partitionDesc.getTabletType()); + copiedTable.getPartitionInfo().setReplicationNum(partitionId, partitionDesc.getReplicationNum()); + copiedTable.getPartitionInfo().setIsInMemory(partitionId, partitionDesc.isInMemory()); + copiedTable.getPartitionInfo().setDataCacheInfo(partitionId, partitionDesc.getDataCacheInfo()); + + Partition partition = + createPartition(db, copiedTable, partitionId, partitionName, version, tabletIdSet, warehouseId); + + partitionList.add(Pair.create(partition, partitionDesc)); + tabletIdSetForAll.addAll(tabletIdSet); + partitionNameToTabletSet.put(partitionName, tabletIdSet); + } + return partitionList; + } + + private void checkIfMetaChange(OlapTable olapTable, OlapTable copiedTable, String tableName) throws DdlException { + // rollup index may be added or dropped during add partition operation. + // schema may be changed during add partition operation. + boolean metaChanged = false; + if (olapTable.getIndexNameToId().size() != copiedTable.getIndexNameToId().size()) { + metaChanged = true; + } else { + // compare schemaHash + for (Map.Entry entry : olapTable.getIndexIdToMeta().entrySet()) { + long indexId = entry.getKey(); + if (!copiedTable.getIndexIdToMeta().containsKey(indexId)) { + metaChanged = true; + break; + } + if (copiedTable.getIndexIdToMeta().get(indexId).getSchemaHash() != + entry.getValue().getSchemaHash()) { + metaChanged = true; + break; + } + } + } + + if (olapTable.getDefaultDistributionInfo().getType() != + copiedTable.getDefaultDistributionInfo().getType()) { + metaChanged = true; + } + + if (metaChanged) { + throw new DdlException("Table[" + tableName + "]'s meta has been changed. try again."); + } + } + + private void updatePartitionInfo(PartitionInfo partitionInfo, List> partitionList, + Set existPartitionNameSet, boolean isTempPartition, + OlapTable olapTable) + throws DdlException { + if (partitionInfo instanceof RangePartitionInfo) { + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; + rangePartitionInfo.handleNewRangePartitionDescs(olapTable.getIdToColumn(), + partitionList, existPartitionNameSet, isTempPartition); + } else if (partitionInfo instanceof ListPartitionInfo) { + ListPartitionInfo listPartitionInfo = (ListPartitionInfo) partitionInfo; + listPartitionInfo.handleNewListPartitionDescs(olapTable.getIdToColumn(), + partitionList, existPartitionNameSet, isTempPartition); + } else { + throw new DdlException("Only support adding partition to range/list partitioned table"); + } + + if (isTempPartition) { + for (Pair entry : partitionList) { + Partition partition = entry.first; + if (!existPartitionNameSet.contains(partition.getName())) { + olapTable.addTempPartition(partition); + } + } + } else { + for (Pair entry : partitionList) { + Partition partition = entry.first; + if (!existPartitionNameSet.contains(partition.getName())) { + olapTable.addPartition(partition); + } + } + } + } + + @Override + public void renamePartition(Database db, Table table, PartitionRenameClause renameClause) throws DdlException { + OlapTable olapTable = (OlapTable) table; + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); + } + + if (!olapTable.getPartitionInfo().isRangePartition()) { + throw new DdlException("Table[" + olapTable.getName() + "] is single partitioned. " + + "no need to rename partition name."); + } + + String partitionName = renameClause.getPartitionName(); + String newPartitionName = renameClause.getNewPartitionName(); + if (partitionName.equalsIgnoreCase(newPartitionName)) { + throw new DdlException("Same partition name"); + } + + Partition partition = olapTable.getPartition(partitionName); + if (partition == null) { + throw new DdlException("Partition[" + partitionName + "] does not exists"); + } + + // check if name is already used + if (olapTable.checkPartitionNameExist(newPartitionName)) { + throw new DdlException("Partition name[" + newPartitionName + "] is already used"); + } + + olapTable.renamePartition(partitionName, newPartitionName); + + // log + TableInfo tableInfo = TableInfo.createForPartitionRename(db.getId(), olapTable.getId(), partition.getId(), + newPartitionName); + GlobalStateMgr.getCurrentState().getLocalMetastore().renamePartition(tableInfo); + LOG.info("rename partition[{}] to {}", partitionName, newPartitionName); + } + + /* + * The entry of replacing partitions with temp partitions. + */ + public void replaceTempPartition(Database db, String tableName, ReplacePartitionClause clause) throws DdlException { + List partitionNames = clause.getPartitionNames(); + // duplicate temp partition will cause Incomplete transaction + List tempPartitionNames = + clause.getTempPartitionNames().stream().distinct().collect(Collectors.toList()); + + boolean isStrictRange = clause.isStrictRange(); + boolean useTempPartitionName = clause.useTempPartitionName(); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + Table table = getTable(db.getFullName(), tableName); + if (table == null) { + ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); + } + + if (!table.isOlapOrCloudNativeTable()) { + throw new DdlException("Table[" + tableName + "] is not OLAP table or LAKE table"); + } + + OlapTable olapTable = (OlapTable) table; + // check partition exist + for (String partName : partitionNames) { + if (!olapTable.checkPartitionNameExist(partName, false)) { + throw new DdlException("Partition[" + partName + "] does not exist"); + } + } + for (String partName : tempPartitionNames) { + if (!olapTable.checkPartitionNameExist(partName, true)) { + throw new DdlException("Temp partition[" + partName + "] does not exist"); + } + } + + partitionNames.stream().forEach(e -> + GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(olapTable.getPartition(e).getId())); + olapTable.replaceTempPartitions(partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); + + // write log + ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(), + partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); + GlobalStateMgr.getCurrentState().getLocalMetastore().replaceTempPartition(info); + LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", + clause.getPartitionNames(), clause.getTempPartitionNames(), tableName); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void setPartitionVersion(AdminSetPartitionVersionStmt stmt) { + Database database = getDb(stmt.getTableName().getDb()); + if (database == null) { + throw ErrorReportException.report(ErrorCode.ERR_BAD_DB_ERROR, stmt.getTableName().getDb()); + } + Locker locker = new Locker(); + locker.lockDatabase(database.getId(), LockType.WRITE); + try { + Table table = getTable(database.getFullName(), stmt.getTableName().getTbl()); + if (table == null) { + throw ErrorReportException.report(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getTableName().getTbl()); + } + if (!table.isOlapTableOrMaterializedView()) { + throw ErrorReportException.report(ErrorCode.ERR_NOT_OLAP_TABLE, stmt.getTableName().getTbl()); + } + + PhysicalPartition physicalPartition; + OlapTable olapTable = (OlapTable) table; + if (stmt.getPartitionId() != -1) { + physicalPartition = olapTable.getPhysicalPartition(stmt.getPartitionId()); + if (physicalPartition == null) { + throw ErrorReportException.report(ErrorCode.ERR_NO_SUCH_PARTITION, stmt.getPartitionName()); + } + } else { + Partition partition = olapTable.getPartition(stmt.getPartitionName()); + if (partition == null) { + throw ErrorReportException.report(ErrorCode.ERR_NO_SUCH_PARTITION, stmt.getPartitionName()); + } + + physicalPartition = partition.getDefaultPhysicalPartition(); + if (partition.getSubPartitions().size() >= 2) { + throw ErrorReportException.report(ErrorCode.ERR_MULTI_SUB_PARTITION, stmt.getPartitionName()); + } + } + + long visibleVersionTime = System.currentTimeMillis(); + physicalPartition.setVisibleVersion(stmt.getVersion(), visibleVersionTime); + physicalPartition.setNextVersion(stmt.getVersion() + 1); + + PartitionVersionRecoveryInfo.PartitionVersion partitionVersion = + new PartitionVersionRecoveryInfo.PartitionVersion(database.getId(), table.getId(), + physicalPartition.getId(), stmt.getVersion()); + for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE)) { + for (Tablet tablet : index.getTablets()) { + if (!(tablet instanceof LocalTablet)) { + continue; + } + + LocalTablet localTablet = (LocalTablet) tablet; + for (Replica replica : localTablet.getAllReplicas()) { + if (replica.getVersion() > stmt.getVersion() && localTablet.getAllReplicas().size() > 1) { + replica.setBad(true); + LOG.warn("set tablet: {} on backend: {} to bad, " + + "because its version: {} is higher than partition visible version: {}", + tablet.getId(), replica.getBackendId(), replica.getVersion(), stmt.getVersion()); + } + } + } + } + GlobalStateMgr.getCurrentState().getLocalMetastore().setPartitionVersion( + new PartitionVersionRecoveryInfo(Lists.newArrayList(partitionVersion), visibleVersionTime)); + LOG.info("Successfully set partition: {} version to {}, table: {}, db: {}", + stmt.getPartitionName(), stmt.getVersion(), table.getName(), database.getFullName()); + } finally { + locker.unLockDatabase(database.getId(), LockType.WRITE); + } + } + + public Partition getPartition(String dbName, String tblName, String partitionName) { + return null; + } + + public Partition getPartition(Database db, OlapTable olapTable, Long partitionId) { + return olapTable.getPartition(partitionId); + } + + public List getAllPartitions(Database db, OlapTable olapTable) { + return new ArrayList<>(olapTable.getAllPartitions()); + } + + public void addSubPartitions(Database db, OlapTable table, Partition partition, + int numSubPartition, long warehouseId) throws DdlException { + try { + table.setAutomaticBucketing(true); + addSubPartitions(db, table, partition, numSubPartition, null, warehouseId); + } finally { + table.setAutomaticBucketing(false); + } + } + + private void addSubPartitions(Database db, OlapTable table, Partition partition, + int numSubPartition, String[] subPartitionNames, long warehouseId) throws DdlException { + OlapTable olapTable; + OlapTable copiedTable; + + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + olapTable = checkTable(db, table.getId()); + + if (partition.getDistributionInfo().getType() != DistributionInfo.DistributionInfoType.RANDOM) { + throw new DdlException("Only support adding physical partition to random distributed table"); + } + + copiedTable = getShadowCopyTable(olapTable); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + + Preconditions.checkNotNull(olapTable); + Preconditions.checkNotNull(copiedTable); + + List subPartitions = new ArrayList<>(); + // create physical partition + for (int i = 0; i < numSubPartition; i++) { + String name = subPartitionNames != null && subPartitionNames.length > i ? subPartitionNames[i] : null; + PhysicalPartition subPartition = createPhysicalPartition(name, db, copiedTable, partition, warehouseId); + subPartitions.add(subPartition); + } + + // build partitions + buildPartitions(db, copiedTable, subPartitions, warehouseId); + + // check again + if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { + throw new DdlException("db " + db.getFullName() + + "(" + db.getId() + ") has been dropped"); + } + try { + // check if meta changed + checkIfMetaChange(olapTable, copiedTable, table.getName()); + + for (PhysicalPartition subPartition : subPartitions) { + // add sub partition + GlobalStateMgr.getCurrentState().getTabletMetastore().addPhysicalPartition(partition, subPartition); + olapTable.addPhysicalPartition(subPartition); + } + + olapTable.setShardGroupChanged(true); + + // add partition log + GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitionLog(db, olapTable, partition, subPartitions); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + private PhysicalPartition createPhysicalPartition(String name, Database db, OlapTable olapTable, + Partition partition, long warehouseId) throws DdlException { + long partitionId = partition.getId(); + DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo().copy(); + olapTable.inferDistribution(distributionInfo); + // create sub partition + Map indexMap = new HashMap<>(); + for (long indexId : olapTable.getIndexIdToMeta().keySet()) { + MaterializedIndex rollup = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); + indexMap.put(indexId, rollup); + } + + Long id = GlobalStateMgr.getCurrentState().getNextId(); + // physical partitions in the same logical partition use the same shard_group_id, + // so that the shards of this logical partition are more evenly distributed. + long shardGroupId = partition.getDefaultPhysicalPartition().getShardGroupId(); + + if (name == null) { + name = partition.generatePhysicalPartitionName(id); + } + PhysicalPartition physicalParition = new PhysicalPartition( + id, name, partition.getId(), shardGroupId, indexMap.get(olapTable.getBaseIndexId())); + + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + short replicationNum = partitionInfo.getReplicationNum(partitionId); + TStorageMedium storageMedium = partitionInfo.getDataProperty(partitionId).getStorageMedium(); + for (Map.Entry entry : indexMap.entrySet()) { + long indexId = entry.getKey(); + MaterializedIndex index = entry.getValue(); + MaterializedIndexMeta indexMeta = olapTable.getIndexIdToMeta().get(indexId); + Set tabletIdSet = new HashSet<>(); + + // create tablets + TabletMeta tabletMeta = + new TabletMeta(db.getId(), olapTable.getId(), id, indexId, indexMeta.getSchemaHash(), + storageMedium, olapTable.isCloudNativeTableOrMaterializedView()); + + if (olapTable.isCloudNativeTableOrMaterializedView()) { + GlobalStateMgr.getCurrentState().getTabletManager().createLakeTablets( + olapTable, id, shardGroupId, index, distributionInfo, + tabletMeta, tabletIdSet, warehouseId); + } else { + GlobalStateMgr.getCurrentState().getTabletManager().createOlapTablets( + olapTable, index, Replica.ReplicaState.NORMAL, distributionInfo, + physicalParition.getVisibleVersion(), replicationNum, tabletMeta, tabletIdSet); + } + if (index.getId() != olapTable.getBaseIndexId()) { + // add rollup index to partition + physicalParition.createRollupIndex(index); + } + } + + return physicalParition; + } + + public void buildPartitions(Database db, OlapTable table, List partitions, long warehouseId) + throws DdlException { + if (partitions.isEmpty()) { + return; + } + int numAliveNodes = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getAliveBackendNumber(); + + if (RunMode.isSharedDataMode()) { + numAliveNodes = 0; + List computeNodeIds = GlobalStateMgr.getCurrentState().getWarehouseMgr().getAllComputeNodeIds(warehouseId); + for (long nodeId : computeNodeIds) { + if (GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getBackendOrComputeNode(nodeId).isAlive()) { + ++numAliveNodes; + } + } + } + if (numAliveNodes == 0) { + if (RunMode.isSharedDataMode()) { + throw new DdlException("no alive compute nodes"); + } else { + throw new DdlException("no alive backends"); + } + } + + int numReplicas = 0; + for (PhysicalPartition partition : partitions) { + numReplicas += partition.storageReplicaCount(); + } + + try { + GlobalStateMgr.getCurrentState().getConsistencyChecker().addCreatingTableId(table.getId()); + if (numReplicas > Config.create_table_max_serial_replicas) { + LOG.info("start to build {} partitions concurrently for table {}.{} with {} replicas", + partitions.size(), db.getFullName(), table.getName(), numReplicas); + TabletTaskExecutor.buildPartitionsConcurrently( + db.getId(), table, partitions, numReplicas, numAliveNodes, warehouseId); + } else { + LOG.info("start to build {} partitions sequentially for table {}.{} with {} replicas", + partitions.size(), db.getFullName(), table.getName(), numReplicas); + TabletTaskExecutor.buildPartitionsSequentially( + db.getId(), table, partitions, numReplicas, numAliveNodes, warehouseId); + } + } finally { + GlobalStateMgr.getCurrentState().getConsistencyChecker().deleteCreatingTableId(table.getId()); + } + } + + // create new partitions from source partitions. + // new partitions have the same indexes as source partitions. + public List createTempPartitionsFromPartitions(Database db, Table table, + String namePostfix, List sourcePartitionIds, + List tmpPartitionIds, DistributionDesc distributionDesc, + long warehouseId) { + Preconditions.checkState(table instanceof OlapTable); + OlapTable olapTable = (OlapTable) table; + Map origPartitions = Maps.newHashMap(); + OlapTable copiedTbl = getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions, distributionDesc != null); + copiedTbl.setDefaultDistributionInfo(olapTable.getDefaultDistributionInfo()); + + // 2. use the copied table to create partitions + List newPartitions = null; + // tabletIdSet to save all newly created tablet ids. + Set tabletIdSet = Sets.newHashSet(); + try { + newPartitions = getNewPartitionsFromPartitions(db, olapTable, sourcePartitionIds, origPartitions, + copiedTbl, namePostfix, tabletIdSet, tmpPartitionIds, distributionDesc, warehouseId); + buildPartitions(db, copiedTbl, newPartitions.stream().map(Partition::getSubPartitions) + .flatMap(p -> p.stream()).collect(Collectors.toList()), warehouseId); + } catch (Exception e) { + // create partition failed, remove all newly created tablets + for (Long tabletId : tabletIdSet) { + GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); + } + LOG.warn("create partitions from partitions failed.", e); + throw new RuntimeException("create partitions failed: " + e.getMessage(), e); + } + return newPartitions; + } + + @VisibleForTesting + public List getNewPartitionsFromPartitions(Database db, OlapTable olapTable, + List sourcePartitionIds, + Map origPartitions, OlapTable copiedTbl, + String namePostfix, Set tabletIdSet, + List tmpPartitionIds, DistributionDesc distributionDesc, + long warehouseId) + throws DdlException { + List newPartitions = Lists.newArrayListWithCapacity(sourcePartitionIds.size()); + for (int i = 0; i < sourcePartitionIds.size(); ++i) { + long newPartitionId = tmpPartitionIds.get(i); + long sourcePartitionId = sourcePartitionIds.get(i); + String newPartitionName = origPartitions.get(sourcePartitionId) + namePostfix; + if (olapTable.checkPartitionNameExist(newPartitionName, true)) { + // to prevent creating the same partitions when failover + // this will happen when OverwriteJob crashed after created temp partitions, + // but before changing to PREPARED state + LOG.warn("partition:{} already exists in table:{}", newPartitionName, olapTable.getName()); + continue; + } + PartitionInfo partitionInfo = copiedTbl.getPartitionInfo(); + partitionInfo.setTabletType(newPartitionId, partitionInfo.getTabletType(sourcePartitionId)); + partitionInfo.setIsInMemory(newPartitionId, partitionInfo.getIsInMemory(sourcePartitionId)); + partitionInfo.setReplicationNum(newPartitionId, partitionInfo.getReplicationNum(sourcePartitionId)); + partitionInfo.setDataProperty(newPartitionId, partitionInfo.getDataProperty(sourcePartitionId)); + if (copiedTbl.isCloudNativeTableOrMaterializedView()) { + partitionInfo.setDataCacheInfo(newPartitionId, partitionInfo.getDataCacheInfo(sourcePartitionId)); + } + + Partition newPartition = null; + if (distributionDesc != null) { + DistributionInfo distributionInfo = distributionDesc.toDistributionInfo(olapTable.getColumns()); + if (distributionInfo.getBucketNum() == 0) { + Partition sourcePartition = olapTable.getPartition(sourcePartitionId); + olapTable.optimizeDistribution(distributionInfo, sourcePartition); + } + newPartition = createPartition( + db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, distributionInfo, warehouseId); + } else { + newPartition = createPartition(db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, warehouseId); + } + + newPartitions.add(newPartition); + } + return newPartitions; + } + + public Partition createPartition(Database db, OlapTable table, long partitionId, String partitionName, + Long version, Set tabletIdSet, long warehouseId) throws DdlException { + DistributionInfo distributionInfo = table.getDefaultDistributionInfo().copy(); + table.inferDistribution(distributionInfo); + + return createPartition(db, table, partitionId, partitionName, version, tabletIdSet, distributionInfo, warehouseId); + } + + public Partition createPartition(Database db, OlapTable table, long partitionId, String partitionName, + Long version, Set tabletIdSet, DistributionInfo distributionInfo, + long warehouseId) throws DdlException { + PartitionInfo partitionInfo = table.getPartitionInfo(); + Map indexMap = new HashMap<>(); + for (long indexId : table.getIndexIdToMeta().keySet()) { + MaterializedIndex rollup = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); + indexMap.put(indexId, rollup); + } + + // create shard group + long shardGroupId = 0; + if (table.isCloudNativeTableOrMaterializedView()) { + shardGroupId = GlobalStateMgr.getCurrentState().getStarOSAgent(). + createShardGroup(db.getId(), table.getId(), partitionId); + } + + Partition logicalPartition = new Partition( + partitionId, + partitionName, + distributionInfo); + + PhysicalPartition physicalPartition = new PhysicalPartition( + partitionId, + partitionName, + partitionId, + shardGroupId, + indexMap.get(table.getBaseIndexId())); + + logicalPartition.addSubPartition(physicalPartition); + + //LogicalPartition partition = new LogicalPartition(partitionId, partitionName, indexMap.get(table.getBaseIndexId()), distributionInfo, shardGroupId); + // version + if (version != null) { + physicalPartition.updateVisibleVersion(version); + } + + short replicationNum = partitionInfo.getReplicationNum(partitionId); + TStorageMedium storageMedium = partitionInfo.getDataProperty(partitionId).getStorageMedium(); + for (Map.Entry entry : indexMap.entrySet()) { + long indexId = entry.getKey(); + MaterializedIndex index = entry.getValue(); + MaterializedIndexMeta indexMeta = table.getIndexIdToMeta().get(indexId); + + // create tablets + TabletMeta tabletMeta = + new TabletMeta(db.getId(), table.getId(), partitionId, indexId, indexMeta.getSchemaHash(), + storageMedium, table.isCloudNativeTableOrMaterializedView()); + + if (table.isCloudNativeTableOrMaterializedView()) { + GlobalStateMgr.getCurrentState().getTabletManager(). + createLakeTablets(table, partitionId, shardGroupId, index, distributionInfo, + tabletMeta, tabletIdSet, warehouseId); + } else { + GlobalStateMgr.getCurrentState().getTabletManager(). + createOlapTablets(table, index, Replica.ReplicaState.NORMAL, distributionInfo, + physicalPartition.getVisibleVersion(), replicationNum, tabletMeta, tabletIdSet); + } + if (index.getId() != table.getBaseIndexId()) { + // add rollup index to partition + physicalPartition.createRollupIndex(index); + } + } + return logicalPartition; + } + + public Database getDbIncludeRecycleBin(long dbId) { + Database db = getDb(dbId); + if (db == null) { + db = GlobalStateMgr.getCurrentState().getRecycleBin().getDatabase(dbId); + } + return db; + } + + public Table getTableIncludeRecycleBin(Database db, long tableId) { + Table table = getTable(db.getId(), tableId); + if (table == null) { + table = GlobalStateMgr.getCurrentState().getRecycleBin().getTable(db.getId(), tableId); + } + return table; + } + + public List
getTablesIncludeRecycleBin(Database db) { + List
tables = db.getTables(); + tables.addAll(GlobalStateMgr.getCurrentState().getRecycleBin().getTables(db.getId())); + return tables; + } + + public Partition getPartitionIncludeRecycleBin(OlapTable table, long partitionId) { + Partition partition = table.getPartition(partitionId); + if (partition == null) { + partition = GlobalStateMgr.getCurrentState().getRecycleBin().getPartition(partitionId); + } + return partition; + } + + public PhysicalPartition getPhysicalPartitionIncludeRecycleBin(OlapTable table, long physicalPartitionId) { + PhysicalPartition partition = table.getPhysicalPartition(physicalPartitionId); + if (partition == null) { + partition = GlobalStateMgr.getCurrentState().getRecycleBin().getPhysicalPartition(physicalPartitionId); + } + return partition; + } + + public Collection getPartitionsIncludeRecycleBin(OlapTable table) { + Collection partitions = new ArrayList<>(table.getPartitions()); + partitions.addAll(GlobalStateMgr.getCurrentState().getRecycleBin().getPartitions(table.getId())); + return partitions; + } + + public Collection getAllPartitionsIncludeRecycleBin(OlapTable table) { + Collection partitions = table.getAllPartitions(); + partitions.addAll(GlobalStateMgr.getCurrentState().getRecycleBin().getPartitions(table.getId())); + return partitions; + } + + // NOTE: result can be null, cause partition erase is not in db lock + public DataProperty getDataPropertyIncludeRecycleBin(PartitionInfo info, long partitionId) { + DataProperty dataProperty = info.getDataProperty(partitionId); + if (dataProperty == null) { + dataProperty = GlobalStateMgr.getCurrentState().getRecycleBin().getPartitionDataProperty(partitionId); + } + return dataProperty; + } + + // NOTE: result can be -1, cause partition erase is not in db lock + public short getReplicationNumIncludeRecycleBin(PartitionInfo info, long partitionId) { + short replicaNum = info.getReplicationNum(partitionId); + if (replicaNum == (short) -1) { + replicaNum = GlobalStateMgr.getCurrentState().getRecycleBin().getPartitionReplicationNum(partitionId); + } + return replicaNum; + } + + public List getDbIdsIncludeRecycleBin() { + List dbIds = getDbIds(); + dbIds.addAll(GlobalStateMgr.getCurrentState().getRecycleBin().getAllDbIds()); + return dbIds; + } + + @Override + public Pair getMaterializedViewIndex(String dbName, String indexName) { + Database database = getDb(dbName); + if (database == null) { + return null; + } + return database.getMaterializedViewIndex(indexName); + } + + @VisibleForTesting + public OlapTable getCopiedTable(Database db, OlapTable olapTable, List sourcePartitionIds, + Map origPartitions, boolean isOptimize) { + OlapTable copiedTbl; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.READ); + try { + if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { + if (!isOptimize || olapTable.getState() != OlapTable.OlapTableState.SCHEMA_CHANGE) { + throw new RuntimeException("Table' state is not NORMAL: " + olapTable.getState() + + ", tableId:" + olapTable.getId() + ", tabletName:" + olapTable.getName()); + } + } + for (Long id : sourcePartitionIds) { + origPartitions.put(id, olapTable.getPartition(id).getName()); + } + copiedTbl = getShadowCopyTable(olapTable); + } finally { + locker.unLockDatabase(db.getId(), LockType.READ); + } + return copiedTbl; + } + + @VisibleForTesting + public OlapTable getCopiedTable(Database db, OlapTable olapTable, List sourcePartitionIds, + Map origPartitions) { + return getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions, false); + } + + private OlapTable getShadowCopyTable(OlapTable olapTable) { + OlapTable copiedTable; + if (olapTable instanceof LakeMaterializedView) { + copiedTable = new LakeMaterializedView(); + } else if (olapTable instanceof MaterializedView) { + copiedTable = new MaterializedView(); + } else if (olapTable instanceof LakeTable) { + copiedTable = new LakeTable(); + } else { + copiedTable = new OlapTable(); + } + + olapTable.copyOnlyForQuery(copiedTable); + return copiedTable; + } + + /* + * generate and check columns' order and key's existence + */ + public void validateColumns(List columns) throws DdlException { + if (columns.isEmpty()) { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_MUST_HAVE_COLUMNS); + } + + boolean encounterValue = false; + boolean hasKey = false; + for (Column column : columns) { + if (column.isKey()) { + if (encounterValue) { + ErrorReport.reportDdlException(ErrorCode.ERR_OLAP_KEY_MUST_BEFORE_VALUE); + } + hasKey = true; + } else { + encounterValue = true; + } + } + + if (!hasKey) { + ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_MUST_HAVE_KEYS); + } + } + + public void setLakeStorageInfo(Database db, OlapTable table, String storageVolumeId, Map properties) + throws DdlException { + DataCacheInfo dataCacheInfo = null; + try { + dataCacheInfo = PropertyAnalyzer.analyzeDataCacheInfo(properties); + } catch (AnalysisException e) { + throw new DdlException(e.getMessage()); + } + + // get service shard storage info from StarMgr + FilePathInfo pathInfo = !storageVolumeId.isEmpty() ? + GlobalStateMgr.getCurrentState().getStarOSAgent().allocateFilePath(storageVolumeId, db.getId(), table.getId()) : + GlobalStateMgr.getCurrentState().getStarOSAgent().allocateFilePath(db.getId(), table.getId()); + table.setStorageInfo(pathInfo, dataCacheInfo); + } + + public static PartitionInfo buildPartitionInfo(CreateMaterializedViewStatement stmt) throws DdlException { + ExpressionPartitionDesc expressionPartitionDesc = stmt.getPartitionExpDesc(); + if (expressionPartitionDesc != null) { + Expr expr = expressionPartitionDesc.getExpr(); + if (expr instanceof SlotRef) { + SlotRef slotRef = (SlotRef) expr; + if (slotRef.getType().getPrimitiveType() == PrimitiveType.VARCHAR) { + return new ListPartitionInfo(PartitionType.LIST, + Collections.singletonList(stmt.getPartitionColumn())); + } + } + if ((expr instanceof FunctionCallExpr)) { + FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr; + if (functionCallExpr.getFnName().getFunction().equalsIgnoreCase(FunctionSet.STR2DATE)) { + Column partitionColumn = new Column(stmt.getPartitionColumn()); + partitionColumn.setType(com.starrocks.catalog.Type.DATE); + return expressionPartitionDesc.toPartitionInfo( + Collections.singletonList(partitionColumn), + Maps.newHashMap(), false); + } + } + return expressionPartitionDesc.toPartitionInfo( + Collections.singletonList(stmt.getPartitionColumn()), + Maps.newHashMap(), false); + } else { + return new SinglePartitionInfo(); + } + } + + public static void inactiveRelatedMaterializedView(Database db, Table olapTable, String reason) { + for (MvId mvId : olapTable.getRelatedMaterializedViews()) { + MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore() + .getTable(db.getId(), mvId.getId()); + if (mv != null) { + LOG.warn("Inactive MV {}/{} because {}", mv.getName(), mv.getId(), reason); + mv.setInactiveAndReason(reason); + + // recursive inactive + inactiveRelatedMaterializedView(db, mv, + MaterializedViewExceptions.inactiveReasonForBaseTableActive(mv.getName())); + } else { + LOG.info("Ignore materialized view {} does not exists", mvId); + } + } + } + + public void onErasePartition(Partition partition) { + // remove tablet in inverted index + TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.deleteTablet(tabletId); + } + } + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TabletHierarchyId.java b/fe/fe-core/src/main/java/com/starrocks/meta/TabletHierarchyId.java new file mode 100644 index 0000000000000..02a7cac9d1f29 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TabletHierarchyId.java @@ -0,0 +1,33 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +public class TabletHierarchyId { + public final long dbId; + public final long tableId; + public final long partitionId; + public final long physicalPartitionId; + public final long materializedIndexId; + public final long tabletId; + + public TabletHierarchyId(long dbId, long tableId, long partitionId, long physicalPartitionId, long materializedIndexId, + long tabletId) { + this.dbId = dbId; + this.tableId = tableId; + this.partitionId = partitionId; + this.physicalPartitionId = physicalPartitionId; + this.materializedIndexId = materializedIndexId; + this.tabletId = tabletId; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TabletInvertedIndexV2.java b/fe/fe-core/src/main/java/com/starrocks/meta/TabletInvertedIndexV2.java new file mode 100644 index 0000000000000..54ca2d16830a8 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TabletInvertedIndexV2.java @@ -0,0 +1,288 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.starrocks.catalog.Replica; +import com.starrocks.catalog.TabletInvertedIndex; +import com.starrocks.catalog.TabletMeta; +import com.starrocks.persist.gson.GsonUtils; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.thrift.TStorageMedium; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +public class TabletInvertedIndexV2 extends TabletInvertedIndex { + public void readLock() { + } + + public void readUnlock() { + } + + public void writeLock() { + } + + public void writeUnlock() { + } + + // tablet id -> tablet meta + // tabletMetaMap/tabletId -> tabletMeta + + // backendId id -> replica list + // backingReplicaMetaTable/backendId/tabletId -> replicaId + + // tablet id -> replica list + // replicaMetaTable/tabletId/replicaId -> Replica + + // replica id -> tablet id + // replicaToTabletMap/replicaId -> tabletId + + @Override + public Long getTabletIdByReplica(long replicaId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("replicaToTabletMap", String.valueOf(replicaId))); + return MetadataHandler.getInstance().get(null, key, Long.class); + } + + @Override + public TabletMeta getTabletMeta(long tabletId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("tabletMetaMap", String.valueOf(tabletId))); + String tabletMetaJson = MetadataHandler.getInstance().get(null, key, String.class); + return GsonUtils.GSON.fromJson(tabletMetaJson, TabletMeta.class); + } + + @Override + public List getTabletMetaList(List tabletIdList) { + List tabletMetaList = new ArrayList<>(tabletIdList.size()); + for (Long tabletId : tabletIdList) { + TabletMeta tabletMeta = getTabletMeta(tabletId); + tabletMetaList.add(Objects.requireNonNullElse(tabletMeta, NOT_EXIST_TABLET_META)); + } + return tabletMetaList; + } + + @Override + public Map getReplicaMetaWithBackend(Long backendId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", String.valueOf(backendId))); + List replicaList = MetadataHandler.getInstance().getPrefix(null, key); + + Map replicaMap = new HashMap<>(); + for (byte[] bytes : replicaList) { + List values = ByteCoder.decode(bytes); + Long tabletId = Long.parseLong(values.get(3)); + Long replicaId = Long.parseLong(values.get(4)); + TabletMeta tabletMeta = getTabletMeta(tabletId); + + Replica replica = getReplicaByTabletMeta(tabletMeta, replicaId); + replicaMap.put(tabletId, replica); + } + + return replicaMap; + } + + @Override + public void addTablet(long tabletId, TabletMeta tabletMeta) { + if (GlobalStateMgr.isCheckpointThread()) { + return; + } + + byte[] key = ByteCoder.encode(Lists.newArrayList("tabletMetaMap", String.valueOf(tabletId))); + MetadataHandler.getInstance().put(null, new String(key), + GsonUtils.GSON.toJson(tabletMeta, TabletMeta.class)); + } + + + @Override + public void deleteTablet(long tabletId) { + if (GlobalStateMgr.isCheckpointThread()) { + return; + } + + TabletMeta tabletMeta = getTabletMeta(tabletId); + List replicas = getReplicaByTabletMeta(tabletMeta); + for (Replica replica : replicas) { + //TODO + } + + byte[] k1 = ByteCoder.encode(Lists.newArrayList("tabletMetaMap", String.valueOf(tabletId))); + MetadataHandler.getInstance().delete(null, k1); + } + + @Override + public void addReplica(long tabletId, Replica replica) { + if (GlobalStateMgr.isCheckpointThread()) { + return; + } + + byte[] k1 = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", + String.valueOf(replica.getBackendId()), String.valueOf(tabletId))); + MetadataHandler.getInstance().put(null, new String(k1), String.valueOf(replica.getId())); + + byte[] k3 = ByteCoder.encode(Lists.newArrayList("replicaToTabletMap", String.valueOf(replica.getId()))); + MetadataHandler.getInstance().put(null, new String(k3), String.valueOf(tabletId)); + } + + @Override + public void deleteReplica(long tabletId, long backendId) { + if (GlobalStateMgr.isCheckpointThread()) { + return; + } + + Long replica = getReplicaId(tabletId, backendId); + + byte[] k1 = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", + String.valueOf(backendId), String.valueOf(tabletId), String.valueOf(replica))); + MetadataHandler.getInstance().delete(null, k1); + + byte[] k3 = ByteCoder.encode(Lists.newArrayList("replicaToTabletMap", + String.valueOf(replica), String.valueOf(tabletId))); + MetadataHandler.getInstance().delete(null, k3); + } + + @Override + public Replica getReplica(long tabletId, long backendId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", + String.valueOf(backendId), + String.valueOf(tabletId))); + List replicaList = MetadataHandler.getInstance().getPrefix(null, key); + + for (byte[] bytes : replicaList) { + List values = ByteCoder.decode(bytes); + Long replicaId = Long.parseLong(values.get(4)); + TabletMeta tabletMeta = getTabletMeta(tabletId); + + Replica replica = getReplicaByTabletMeta(tabletMeta, replicaId); + return replica; + } + + return null; + } + + public Long getReplicaId(long tabletId, long backendId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", + String.valueOf(backendId), + String.valueOf(tabletId))); + List replicaList = MetadataHandler.getInstance().getPrefix(null, key); + + for (byte[] bytes : replicaList) { + List values = ByteCoder.decode(bytes); + Long replicaId = Long.parseLong(values.get(4)); + return replicaId; + } + + return null; + } + + private Replica getReplicaByTabletMeta(TabletMeta tabletMeta, long replicaId) { + return null; + } + + private List getReplicaByTabletMeta(TabletMeta tabletMeta) { + return null; + } + + @Override + public List getReplicasByTabletId(long tabletId) { + TabletMeta tabletMeta = getTabletMeta(tabletId); + return getReplicaByTabletMeta(tabletMeta); + } + + @Override + public List getReplicasOnBackendByTabletIds(List tabletIds, long backendId) { + + List replicas = new ArrayList<>(); + for (Long tabletId : tabletIds) { + Replica replica = getReplica(tabletId, backendId); + replicas.add(replica); + } + + return replicas; + } + + @Override + public List getTabletIdsByBackendId(long backendId) { + byte[] key = ByteCoder.encode(Lists.newArrayList("backingReplicaMetaTable", String.valueOf(backendId))); + List bytesList = MetadataHandler.getInstance().getPrefix(null, key); + + Set tabletIds = new HashSet<>(); + for (byte[] bytes : bytesList) { + List values = ByteCoder.decode(bytes); + Long tabletId = Long.valueOf(values.get(3)); + tabletIds.add(tabletId); + } + + return new ArrayList<>(tabletIds); + } + + @Override + public List getTabletIdsByBackendIdAndStorageMedium(long backendId, TStorageMedium storageMedium) { + List tabletIds = getTabletIdsByBackendId(backendId); + + List t = new ArrayList<>(); + for (Long tabletId : tabletIds) { + TabletMeta tabletMeta = getTabletMeta(tabletId); + if (tabletMeta.getStorageMedium() == storageMedium) { + t.add(tabletId); + } + } + + return t; + } + + @Override + public long getTabletNumByBackendId(long backendId) { + return getTabletIdsByBackendId(backendId).size(); + } + + @Override + public long getTabletNumByBackendIdAndPathHash(long backendId, long pathHash) { + Collection replicas = getReplicaMetaWithBackend(backendId).values(); + int count = 0; + for (Replica replica : replicas) { + if (replica.getPathHash() == pathHash) { + count++; + } + } + + return count; + } + + @Override + public Map getReplicaNumByBeIdAndStorageMedium(long backendId) { + Map replicaNumMap = Maps.newHashMap(); + long hddNum = 0; + long ssdNum = 0; + + List tabletIds = getTabletIdsByBackendId(backendId); + for (Long tabletId : tabletIds) { + TabletMeta tabletMeta = getTabletMeta(tabletId); + if (tabletMeta.getStorageMedium() == TStorageMedium.HDD) { + hddNum++; + } else { + ssdNum++; + } + } + + replicaNumMap.put(TStorageMedium.HDD, hddNum); + replicaNumMap.put(TStorageMedium.SSD, ssdNum); + return replicaNumMap; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TabletManager.java b/fe/fe-core/src/main/java/com/starrocks/meta/TabletManager.java new file mode 100644 index 0000000000000..6b793a7e10513 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TabletManager.java @@ -0,0 +1,539 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; +import com.starrocks.catalog.ColocateTableIndex; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.DistributionInfo; +import com.starrocks.catalog.LocalTablet; +import com.starrocks.catalog.MaterializedIndex; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.Replica; +import com.starrocks.catalog.Tablet; +import com.starrocks.catalog.TabletInvertedIndex; +import com.starrocks.catalog.TabletMeta; +import com.starrocks.common.Config; +import com.starrocks.common.DdlException; +import com.starrocks.common.ErrorCode; +import com.starrocks.common.ErrorReportException; +import com.starrocks.common.Pair; +import com.starrocks.common.util.concurrent.CountingLatch; +import com.starrocks.common.util.concurrent.lock.LockType; +import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.lake.LakeTablet; +import com.starrocks.persist.BackendTabletsInfo; +import com.starrocks.persist.BatchDeleteReplicaInfo; +import com.starrocks.persist.ColocatePersistInfo; +import com.starrocks.persist.ReplicaPersistInfo; +import com.starrocks.persist.SetReplicaStatusOperationLog; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.server.WarehouseManager; +import com.starrocks.sql.ast.AdminCheckTabletsStmt; +import com.starrocks.sql.ast.AdminSetReplicaStatusStmt; +import com.starrocks.system.SystemInfoService; +import com.starrocks.thrift.TStorageMedium; +import com.starrocks.warehouse.Warehouse; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +public class TabletManager { + private static final Logger LOG = LogManager.getLogger(TabletManager.class); + /** + * Concurrent colocate table creation process have dependency on each other + * (even in different databases), but we do not want to affect the performance + * of non-colocate table creation, so here we use a separate latch to + * synchronize only the creation of colocate tables. + */ + private final CountingLatch colocateTableCreateSyncer = new CountingLatch(0); + + public void createLakeTablets(OlapTable table, long partitionId, long shardGroupId, MaterializedIndex index, + DistributionInfo distributionInfo, TabletMeta tabletMeta, + Set tabletIdSet, long warehouseId) + throws DdlException { + Preconditions.checkArgument(table.isCloudNativeTableOrMaterializedView()); + + DistributionInfo.DistributionInfoType distributionInfoType = distributionInfo.getType(); + if (distributionInfoType != DistributionInfo.DistributionInfoType.HASH + && distributionInfoType != DistributionInfo.DistributionInfoType.RANDOM) { + throw new DdlException("Unknown distribution type: " + distributionInfoType); + } + + Map properties = new HashMap<>(); + properties.put(LakeTablet.PROPERTY_KEY_TABLE_ID, Long.toString(table.getId())); + properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(partitionId)); + properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(index.getId())); + int bucketNum = distributionInfo.getBucketNum(); + WarehouseManager warehouseManager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); + Optional workerGroupId = warehouseManager.selectWorkerGroupByWarehouseId(warehouseId); + if (workerGroupId.isEmpty()) { + Warehouse warehouse = warehouseManager.getWarehouse(warehouseId); + throw ErrorReportException.report(ErrorCode.ERR_NO_NODES_IN_WAREHOUSE, warehouse.getName()); + } + List shardIds = GlobalStateMgr.getCurrentState().getStarOSAgent().createShards(bucketNum, + table.getPartitionFilePathInfo(partitionId), table.getPartitionFileCacheInfo(partitionId), shardGroupId, + null, properties, workerGroupId.get()); + for (long shardId : shardIds) { + Tablet tablet = new LakeTablet(shardId); + GlobalStateMgr.getCurrentState().getTabletMetastore().addTablet(index, tablet, tabletMeta); + tabletIdSet.add(tablet.getId()); + } + } + + public void createOlapTablets(OlapTable table, MaterializedIndex index, Replica.ReplicaState replicaState, + DistributionInfo distributionInfo, long version, short replicationNum, + TabletMeta tabletMeta, Set tabletIdSet) throws DdlException { + Preconditions.checkArgument(replicationNum > 0); + + ColocateTableIndex colocateTableIndex = GlobalStateMgr.getCurrentState().getColocateTableIndex(); + + DistributionInfo.DistributionInfoType distributionInfoType = distributionInfo.getType(); + if (distributionInfoType != DistributionInfo.DistributionInfoType.HASH + && distributionInfoType != DistributionInfo.DistributionInfoType.RANDOM) { + throw new DdlException("Unknown distribution type: " + distributionInfoType); + } + + List> backendsPerBucketSeq = null; + ColocateTableIndex.GroupId groupId = null; + boolean initBucketSeqWithSameOrigNameGroup = false; + boolean isColocateTable = colocateTableIndex.isColocateTable(tabletMeta.getTableId()); + // chooseBackendsArbitrary is true, means this may be the first table of colocation group, + // or this is just a normal table, and we can choose backends arbitrary. + // otherwise, backends should be chosen from backendsPerBucketSeq; + boolean chooseBackendsArbitrary; + + // We should synchronize the creation of colocate tables, otherwise it can have concurrent issues. + // Considering the following situation, + // T1: P1 issues `create colocate table` and finds that there isn't a bucket sequence associated + // with the colocate group, so it will initialize the bucket sequence for the first time + // T2: P2 do the same thing as P1 + // T3: P1 set the bucket sequence for colocate group stored in `ColocateTableIndex` + // T4: P2 also set the bucket sequence, hence overwrite what P1 just wrote + // T5: After P1 creates the colocate table, the actual tablet distribution won't match the bucket sequence + // of the colocate group, and balancer will create a lot of COLOCATE_MISMATCH tasks which shouldn't exist. + if (isColocateTable) { + try { + // Optimization: wait first time, before global lock + colocateTableCreateSyncer.awaitZero(); + // Since we have supported colocate tables in different databases, + // we should use global lock, not db lock. + GlobalStateMgr.getCurrentState().tryLock(false); + try { + // Wait again, for safety + // We are in global lock, we should have timeout in case holding lock for too long + colocateTableCreateSyncer.awaitZero(Config.catalog_try_lock_timeout_ms, TimeUnit.MILLISECONDS); + // if this is a colocate table, try to get backend seqs from colocation index. + groupId = colocateTableIndex.getGroup(tabletMeta.getTableId()); + backendsPerBucketSeq = colocateTableIndex.getBackendsPerBucketSeq(groupId); + if (backendsPerBucketSeq.isEmpty()) { + List colocateWithGroupsInOtherDb = + colocateTableIndex.getColocateWithGroupsInOtherDb(groupId); + if (!colocateWithGroupsInOtherDb.isEmpty()) { + backendsPerBucketSeq = + colocateTableIndex.getBackendsPerBucketSeq(colocateWithGroupsInOtherDb.get(0)); + initBucketSeqWithSameOrigNameGroup = true; + } + } + chooseBackendsArbitrary = backendsPerBucketSeq == null || backendsPerBucketSeq.isEmpty(); + if (chooseBackendsArbitrary) { + colocateTableCreateSyncer.increment(); + } + } finally { + GlobalStateMgr.getCurrentState().unlock(); + } + } catch (InterruptedException e) { + LOG.warn("wait for concurrent colocate table creation finish failed, msg: {}", + e.getMessage(), e); + Thread.currentThread().interrupt(); + throw new DdlException("wait for concurrent colocate table creation finish failed", e); + } + } else { + chooseBackendsArbitrary = true; + } + + try { + if (chooseBackendsArbitrary) { + backendsPerBucketSeq = Lists.newArrayList(); + } + for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { + // create a new tablet with random chosen backends + LocalTablet tablet = new LocalTablet(GlobalStateMgr.getCurrentState().getNextId()); + + // add tablet to inverted index first + GlobalStateMgr.getCurrentState().getTabletMetastore().addTablet(index, tablet, tabletMeta); + tabletIdSet.add(tablet.getId()); + + // get BackendIds + List chosenBackendIds; + if (chooseBackendsArbitrary) { + // This is the first colocate table in the group, or just a normal table, + // randomly choose backends + if (Config.enable_strict_storage_medium_check) { + chosenBackendIds = + chosenBackendIdBySeq(replicationNum, table.getLocation(), tabletMeta.getStorageMedium()); + } else { + try { + chosenBackendIds = chosenBackendIdBySeq(replicationNum, table.getLocation()); + } catch (DdlException ex) { + throw new DdlException(String.format("%s, table=%s, default_replication_num=%d", + ex.getMessage(), table.getName(), Config.default_replication_num)); + } + } + backendsPerBucketSeq.add(chosenBackendIds); + } else { + // get backends from existing backend sequence + chosenBackendIds = backendsPerBucketSeq.get(i); + } + + // create replicas + for (long backendId : chosenBackendIds) { + long replicaId = GlobalStateMgr.getCurrentState().getNextId(); + Replica replica = new Replica(replicaId, backendId, replicaState, version, + tabletMeta.getOldSchemaHash()); + tablet.addReplica(replica); + } + Preconditions.checkState(chosenBackendIds.size() == replicationNum, + chosenBackendIds.size() + " vs. " + replicationNum); + } + + // In the following two situations, we should set the bucket seq for colocate group and persist the info, + // 1. This is the first time we add a table to colocate group, and it doesn't have the same original name + // with colocate group in other database. + // 2. It's indeed the first time, but it should colocate with group in other db + // (because of having the same original name), we should use the bucket + // seq of other group to initialize our own. + if ((groupId != null && chooseBackendsArbitrary) || initBucketSeqWithSameOrigNameGroup) { + colocateTableIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeq); + ColocatePersistInfo info = + ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, backendsPerBucketSeq); + GlobalStateMgr.getCurrentState().getEditLog().logColocateBackendsPerBucketSeq(info); + } + } finally { + if (isColocateTable && chooseBackendsArbitrary) { + colocateTableCreateSyncer.decrement(); + } + } + } + + // create replicas for tablet with random chosen backends + private List chosenBackendIdBySeq(int replicationNum, Multimap locReq, + TStorageMedium storageMedium) + throws DdlException { + List chosenBackendIds = + GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getNodeSelector() + .seqChooseBackendIdsByStorageMedium(replicationNum, + true, true, locReq, storageMedium); + if (CollectionUtils.isEmpty(chosenBackendIds)) { + throw new DdlException( + "Failed to find enough hosts with storage medium " + storageMedium + + " at all backends, number of replicas needed: " + + replicationNum + ". Storage medium check failure can be forcefully ignored by executing " + + "'ADMIN SET FRONTEND CONFIG (\"enable_strict_storage_medium_check\" = \"false\");', " + + "but incompatible medium type can cause balance problem, so we strongly recommend" + + " creating table with compatible 'storage_medium' property set."); + } + return chosenBackendIds; + } + + private List chosenBackendIdBySeq(int replicationNum, Multimap locReq) throws DdlException { + SystemInfoService systemInfoService = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); + List chosenBackendIds = systemInfoService.getNodeSelector() + .seqChooseBackendIds(replicationNum, true, true, locReq); + if (!CollectionUtils.isEmpty(chosenBackendIds)) { + return chosenBackendIds; + } else if (replicationNum > 1) { + List backendIds = systemInfoService.getBackendIds(true); + throw new DdlException( + String.format("Table replication num should be less than or equal to the number of available BE nodes. " + + "You can change this default by setting the replication_num table properties. " + + "Current alive backend is [%s]. ", Joiner.on(",").join(backendIds))); + } else { + throw new DdlException("No alive nodes"); + } + } + + // entry of checking tablets operation + public void checkTablets(AdminCheckTabletsStmt stmt) { + AdminCheckTabletsStmt.CheckType type = stmt.getType(); + if (type == AdminCheckTabletsStmt.CheckType.CONSISTENCY) { + GlobalStateMgr.getCurrentState().getConsistencyChecker().addTabletsToCheck(stmt.getTabletIds()); + } + } + + // Set specified replica's status. If replica does not exist, just ignore it. + public void setReplicaStatus(AdminSetReplicaStatusStmt stmt) { + long tabletId = stmt.getTabletId(); + long backendId = stmt.getBackendId(); + Replica.ReplicaStatus status = stmt.getStatus(); + setReplicaStatusInternal(tabletId, backendId, status, false); + } + + public void replaySetReplicaStatus(SetReplicaStatusOperationLog log) { + setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true); + } + + private void setReplicaStatusInternal(long tabletId, long backendId, Replica.ReplicaStatus status, + boolean isReplay) { + TabletInvertedIndex tabletInvertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + TabletMeta meta = tabletInvertedIndex.getTabletMeta(tabletId); + if (meta == null) { + LOG.info("tablet {} does not exist", tabletId); + return; + } + long dbId = meta.getDbId(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); + if (db == null) { + LOG.info("database {} of tablet {} does not exist", dbId, tabletId); + return; + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + Replica replica = tabletInvertedIndex.getReplica(tabletId, backendId); + if (replica == null) { + LOG.info("replica of tablet {} does not exist", tabletId); + return; + } + if (status == Replica.ReplicaStatus.BAD || status == Replica.ReplicaStatus.OK) { + if (replica.setBadForce(status == Replica.ReplicaStatus.BAD)) { + if (!isReplay) { + // Put this tablet into urgent table so that it can be repaired ASAP. + GlobalStateMgr.getCurrentState().getTabletChecker() + .setTabletForUrgentRepair(dbId, meta.getTableId(), meta.getPartitionId()); + SetReplicaStatusOperationLog log = + new SetReplicaStatusOperationLog(backendId, tabletId, status); + GlobalStateMgr.getCurrentState().getEditLog().logSetReplicaStatus(log); + } + LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}", + replica.getId(), tabletId, backendId, status, isReplay); + } + } + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void replayAddReplica(ReplicaPersistInfo info) { + StarRocksMeta starRocksMeta = GlobalStateMgr.getServingState().getStarRocksMeta(); + + Database db = starRocksMeta.getDbIncludeRecycleBin(info.getDbId()); + if (db == null) { + LOG.warn("replay add replica failed, db is null, info: {}", info); + return; + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) starRocksMeta.getTableIncludeRecycleBin(db, info.getTableId()); + if (olapTable == null) { + LOG.warn("replay add replica failed, table is null, info: {}", info); + return; + } + PhysicalPartition partition = starRocksMeta + .getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); + if (partition == null) { + LOG.warn("replay add replica failed, partition is null, info: {}", info); + return; + } + MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); + if (materializedIndex == null) { + LOG.warn("replay add replica failed, materializedIndex is null, info: {}", info); + return; + } + LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); + if (tablet == null) { + LOG.warn("replay add replica failed, tablet is null, info: {}", info); + return; + } + + // for compatibility + int schemaHash = info.getSchemaHash(); + if (schemaHash == -1) { + schemaHash = olapTable.getSchemaHashByIndexId(info.getIndexId()); + } + + Replica replica = new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), + schemaHash, info.getDataSize(), info.getRowCount(), + Replica.ReplicaState.NORMAL, + info.getLastFailedVersion(), + info.getLastSuccessVersion(), + info.getMinReadableVersion()); + tablet.addReplica(replica); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void replayUpdateReplica(ReplicaPersistInfo info) { + StarRocksMeta starRocksMeta = GlobalStateMgr.getServingState().getStarRocksMeta(); + + Database db = starRocksMeta.getDbIncludeRecycleBin(info.getDbId()); + if (db == null) { + LOG.warn("replay update replica failed, db is null, info: {}", info); + return; + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) starRocksMeta.getTableIncludeRecycleBin(db, info.getTableId()); + if (olapTable == null) { + LOG.warn("replay update replica failed, table is null, info: {}", info); + return; + } + PhysicalPartition partition = starRocksMeta + .getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); + if (partition == null) { + LOG.warn("replay update replica failed, partition is null, info: {}", info); + return; + } + MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); + if (materializedIndex == null) { + LOG.warn("replay update replica failed, materializedIndex is null, info: {}", info); + return; + } + LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); + if (tablet == null) { + LOG.warn("replay update replica failed, tablet is null, info: {}", info); + return; + } + Replica replica = tablet.getReplicaByBackendId(info.getBackendId()); + if (replica == null) { + LOG.warn("replay update replica failed, replica is null, info: {}", info); + return; + } + replica.updateRowCount(info.getVersion(), info.getMinReadableVersion(), info.getDataSize(), info.getRowCount()); + replica.setBad(false); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void replayDeleteReplica(ReplicaPersistInfo info) { + StarRocksMeta starRocksMeta = GlobalStateMgr.getServingState().getStarRocksMeta(); + + Database db = starRocksMeta.getDbIncludeRecycleBin(info.getDbId()); + if (db == null) { + LOG.warn("replay delete replica failed, db is null, info: {}", info); + return; + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) starRocksMeta.getTableIncludeRecycleBin(db, info.getTableId()); + if (olapTable == null) { + LOG.warn("replay delete replica failed, table is null, info: {}", info); + return; + } + PhysicalPartition partition = starRocksMeta + .getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); + if (partition == null) { + LOG.warn("replay delete replica failed, partition is null, info: {}", info); + return; + } + MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); + if (materializedIndex == null) { + LOG.warn("replay delete replica failed, materializedIndex is null, info: {}", info); + return; + } + LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); + if (tablet == null) { + LOG.warn("replay delete replica failed, tablet is null, info: {}", info); + return; + } + tablet.deleteReplicaByBackendId(info.getBackendId()); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + public void replayBatchDeleteReplica(BatchDeleteReplicaInfo info) { + if (info.getReplicaInfoList() != null) { + for (ReplicaPersistInfo persistInfo : info.getReplicaInfoList()) { + replayDeleteReplica(persistInfo); + } + } else { + LOG.warn("invalid BatchDeleteReplicaInfo, replicaInfoList is null"); + } + } + + public void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { + List> tabletsWithSchemaHash = backendTabletsInfo.getTabletSchemaHash(); + if (!tabletsWithSchemaHash.isEmpty()) { + // In previous version, we save replica info in `tabletsWithSchemaHash`, + // but it is wrong because we can not get replica from `tabletInvertedIndex` when doing checkpoint, + // because when doing checkpoint, the tabletInvertedIndex is not initialized at all. + // + // So we can only discard this information, in this case, it is equivalent to losing the record of these operations. + // But it doesn't matter, these records are currently only used to record whether a replica is in a bad state. + // This state has little effect on the system, and it can be restored after the system has processed the bad state replica. + for (Pair tabletInfo : tabletsWithSchemaHash) { + LOG.warn("find an old backendTabletsInfo for tablet {}, ignore it", tabletInfo.first); + } + return; + } + + // in new version, replica info is saved here. + // but we need to get replica from db->tbl->partition->... + List replicaPersistInfos = backendTabletsInfo.getReplicaPersistInfos(); + for (ReplicaPersistInfo info : replicaPersistInfos) { + long dbId = info.getDbId(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); + if (db == null) { + continue; + } + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore() + .getTable(db.getId(), info.getTableId()); + if (tbl == null) { + continue; + } + Partition partition = tbl.getPartition(info.getPartitionId()); + if (partition == null) { + continue; + } + MaterializedIndex mindex = partition.getDefaultPhysicalPartition().getIndex(info.getIndexId()); + if (mindex == null) { + continue; + } + LocalTablet tablet = (LocalTablet) mindex.getTablet(info.getTabletId()); + if (tablet == null) { + continue; + } + Replica replica = tablet.getReplicaById(info.getReplicaId()); + if (replica != null) { + replica.setBad(true); + LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying", + info.getReplicaId(), info.getTabletId(), info.getBackendId()); + } + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastore.java b/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastore.java new file mode 100644 index 0000000000000..fb0c0196772f3 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastore.java @@ -0,0 +1,92 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.starrocks.catalog.LocalTablet; +import com.starrocks.catalog.MaterializedIndex; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.Replica; +import com.starrocks.catalog.Tablet; +import com.starrocks.catalog.TabletMeta; +import com.starrocks.persist.ReplicaPersistInfo; +import com.starrocks.server.GlobalStateMgr; + +import java.util.ArrayList; +import java.util.List; + +public class TabletMetastore implements TabletMetastoreInterface { + public List getAllPhysicalPartition(Partition partition) { + return new ArrayList<>(partition.getSubPartitions()); + } + + @Override + public PhysicalPartition getPhysicalPartition(Partition partition, Long physicalPartitionId) { + return partition.getSubPartition(physicalPartitionId); + } + + public void addPhysicalPartition(Partition partition, PhysicalPartition physicalPartition) { + partition.addSubPartition(physicalPartition); + } + + public void dropPhysicalPartition(Partition partition, Long physicalPartitionId) { + partition.removeSubPartition(physicalPartitionId); + } + + public List getMaterializedIndices(PhysicalPartition physicalPartition, + MaterializedIndex.IndexExtState indexExtState) { + return physicalPartition.getMaterializedIndices(indexExtState); + } + + public MaterializedIndex getMaterializedIndex(PhysicalPartition physicalPartition, Long mIndexId) { + return physicalPartition.getIndex(mIndexId); + } + + public void addMaterializedIndex(PhysicalPartition physicalPartition, MaterializedIndex materializedIndex) { + physicalPartition.createRollupIndex(materializedIndex); + } + + public void dropMaterializedIndex(PhysicalPartition physicalPartition, Long mIndexId) { + physicalPartition.deleteRollupIndex(mIndexId); + } + + public List getAllTablets(MaterializedIndex materializedIndex) { + return materializedIndex.getTablets(); + } + + public List getAllTabletIDs(MaterializedIndex materializedIndex) { + return materializedIndex.getTabletIdsInOrder(); + } + + public Tablet getTablet(MaterializedIndex materializedIndex, Long tabletId) { + return materializedIndex.getTablet(tabletId); + } + + public void addTablet(MaterializedIndex materializedIndex, Tablet tablet, TabletMeta tabletMeta) { + materializedIndex.addTablet(tablet, tabletMeta); + } + + public List getAllReplicas(Tablet tablet) { + return tablet.getAllReplicas(); + } + + public Replica getReplica(LocalTablet tablet, Long replicaId) { + return tablet.getReplicaById(replicaId); + } + + @Override + public void addReplica(ReplicaPersistInfo replicaPersistInfo, Tablet tablet, Replica replica) { + GlobalStateMgr.getCurrentState().getEditLog().logAddReplica(replicaPersistInfo); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastoreInterface.java b/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastoreInterface.java new file mode 100644 index 0000000000000..34b5fe5834ce5 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TabletMetastoreInterface.java @@ -0,0 +1,60 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.starrocks.meta; + +import com.starrocks.catalog.LocalTablet; +import com.starrocks.catalog.MaterializedIndex; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; +import com.starrocks.catalog.Replica; +import com.starrocks.catalog.Tablet; +import com.starrocks.catalog.TabletMeta; +import com.starrocks.persist.ReplicaPersistInfo; + +import java.util.List; + +public interface TabletMetastoreInterface { + List getAllPhysicalPartition(Partition partition); + + + PhysicalPartition getPhysicalPartition(Partition partition, Long physicalPartitionId); + + void addPhysicalPartition(Partition partition, PhysicalPartition physicalPartition); + + void dropPhysicalPartition(Partition partition, Long physicalPartitionId); + + List getMaterializedIndices(PhysicalPartition physicalPartition, + MaterializedIndex.IndexExtState indexExtState); + + MaterializedIndex getMaterializedIndex(PhysicalPartition physicalPartition, Long mIndexId); + + void addMaterializedIndex(PhysicalPartition physicalPartition, MaterializedIndex materializedIndex); + + void dropMaterializedIndex(PhysicalPartition physicalPartition, Long mIndexId); + + List getAllTablets(MaterializedIndex materializedIndex); + + List getAllTabletIDs(MaterializedIndex materializedIndex); + + Tablet getTablet(MaterializedIndex materializedIndex, Long tabletId); + + void addTablet(MaterializedIndex materializedIndex, Tablet tablet, TabletMeta tabletMeta); + + List getAllReplicas(Tablet tablet); + + Replica getReplica(LocalTablet tablet, Long replicaId); + + void addReplica(ReplicaPersistInfo replicaPersistInfo, Tablet tablet, Replica replica); +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/TxnMeta.java b/fe/fe-core/src/main/java/com/starrocks/meta/TxnMeta.java new file mode 100644 index 0000000000000..b03e6130601bf --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/TxnMeta.java @@ -0,0 +1,24 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +import com.sleepycat.je.Transaction; + +public class TxnMeta { + private Transaction transaction; + + public void setTransaction(Transaction transaction) { + this.transaction = transaction; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/meta/VersionManager.java b/fe/fe-core/src/main/java/com/starrocks/meta/VersionManager.java new file mode 100644 index 0000000000000..c21c15850797f --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/meta/VersionManager.java @@ -0,0 +1,22 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.meta; + +public class VersionManager { + public void setDataVersion(long partitionId, long dataVersion) { + //OlapTable table; + //Partition partition = table.getPartition(partitionId); + //partition.setDataVersion(dataVersion); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/CreateDbInfo.java b/fe/fe-core/src/main/java/com/starrocks/persist/CreateDbInfo.java index ae0cb774b1d30..c2bb4ca6c042f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/CreateDbInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/CreateDbInfo.java @@ -17,13 +17,14 @@ import com.google.gson.annotations.SerializedName; import com.starrocks.common.io.Text; import com.starrocks.common.io.Writable; +import com.starrocks.meta.TxnMeta; import com.starrocks.persist.gson.GsonUtils; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -public class CreateDbInfo implements Writable { +public class CreateDbInfo extends TxnMeta implements Writable { @SerializedName(value = "i") private long id; diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/DatabaseInfo.java b/fe/fe-core/src/main/java/com/starrocks/persist/DatabaseInfo.java index 7ab02e6d9359e..1fd38a4b173ad 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/DatabaseInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/DatabaseInfo.java @@ -35,16 +35,18 @@ package com.starrocks.persist; import com.google.gson.annotations.SerializedName; +import com.sleepycat.je.Transaction; import com.starrocks.cluster.ClusterNamespace; import com.starrocks.common.io.Text; import com.starrocks.common.io.Writable; +import com.starrocks.meta.TxnMeta; import com.starrocks.sql.ast.AlterDatabaseQuotaStmt.QuotaType; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -public class DatabaseInfo implements Writable { +public class DatabaseInfo extends TxnMeta implements Writable { @SerializedName("db") private String dbName; @@ -130,4 +132,5 @@ public QuotaType getQuotaType() { return quotaType; } + public Transaction transaction; } diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java index a1889b354bbff..24a527521ec86 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java @@ -120,6 +120,7 @@ import org.apache.logging.log4j.Logger; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -194,7 +195,7 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) } case OperationType.OP_ERASE_DB: { Text dbId = (Text) journal.getData(); - globalStateMgr.getLocalMetastore().replayEraseDatabase(Long.parseLong(dbId.toString())); + globalStateMgr.getRecycleBin().replayEraseDatabase(Long.parseLong(dbId.toString())); break; } case OperationType.OP_RECOVER_DB: @@ -290,33 +291,34 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) } case OperationType.OP_ERASE_TABLE: { Text tableId = (Text) journal.getData(); - globalStateMgr.getLocalMetastore().replayEraseTable(Long.parseLong(tableId.toString())); + globalStateMgr.getRecycleBin().replayEraseTable( + Collections.singletonList(Long.parseLong(tableId.toString()))); break; } case OperationType.OP_ERASE_MULTI_TABLES: { MultiEraseTableInfo multiEraseTableInfo = (MultiEraseTableInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayEraseMultiTables(multiEraseTableInfo); + globalStateMgr.getRecycleBin().replayEraseTable(multiEraseTableInfo.getTableIds()); break; } case OperationType.OP_DISABLE_TABLE_RECOVERY: { DisableTableRecoveryInfo disableTableRecoveryInfo = (DisableTableRecoveryInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayDisableTableRecovery(disableTableRecoveryInfo); + globalStateMgr.getRecycleBin().replayDisableTableRecovery(disableTableRecoveryInfo.getTableIds()); break; } case OperationType.OP_DISABLE_PARTITION_RECOVERY: { DisablePartitionRecoveryInfo disableRecoveryInfo = (DisablePartitionRecoveryInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayDisablePartitionRecovery(disableRecoveryInfo); + globalStateMgr.getRecycleBin().replayDisablePartitionRecovery(disableRecoveryInfo.getPartitionId()); break; } case OperationType.OP_ERASE_PARTITION: { Text partitionId = (Text) journal.getData(); - globalStateMgr.getLocalMetastore().replayErasePartition(Long.parseLong(partitionId.toString())); + globalStateMgr.getRecycleBin().replayErasePartition(Long.parseLong(partitionId.toString())); break; } case OperationType.OP_RECOVER_TABLE: case OperationType.OP_RECOVER_TABLE_V2: { RecoverInfo info = (RecoverInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayRecoverTable(info); + globalStateMgr.getRecycleBin().replayRecoverTable(info); break; } case OperationType.OP_RECOVER_PARTITION: @@ -452,24 +454,24 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) case OperationType.OP_ADD_REPLICA: case OperationType.OP_ADD_REPLICA_V2: { ReplicaPersistInfo info = (ReplicaPersistInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayAddReplica(info); + globalStateMgr.getTabletManager().replayAddReplica(info); break; } case OperationType.OP_UPDATE_REPLICA: case OperationType.OP_UPDATE_REPLICA_V2: { ReplicaPersistInfo info = (ReplicaPersistInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayUpdateReplica(info); + globalStateMgr.getTabletManager().replayUpdateReplica(info); break; } case OperationType.OP_DELETE_REPLICA: case OperationType.OP_DELETE_REPLICA_V2: { ReplicaPersistInfo info = (ReplicaPersistInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayDeleteReplica(info); + globalStateMgr.getTabletManager().replayDeleteReplica(info); break; } case OperationType.OP_BATCH_DELETE_REPLICA: { BatchDeleteReplicaInfo info = (BatchDeleteReplicaInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayBatchDeleteReplica(info); + globalStateMgr.getTabletManager().replayBatchDeleteReplica(info); break; } case OperationType.OP_ADD_COMPUTE_NODE: { @@ -576,11 +578,6 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) globalStateMgr.getLoadInstance().setLoadErrorHubInfo(param); break; } - case OperationType.OP_UPDATE_CLUSTER_AND_BACKENDS: { - final BackendIdsUpdateInfo info = (BackendIdsUpdateInfo) journal.getData(); - globalStateMgr.replayUpdateClusterAndBackends(info); - break; - } case OperationType.OP_UPSERT_TRANSACTION_STATE_V2: { final TransactionState state = (TransactionState) journal.getData(); GlobalStateMgr.getCurrentState().getGlobalTransactionMgr().replayUpsertTransactionState(state); @@ -673,7 +670,7 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) case OperationType.OP_BACKEND_TABLETS_INFO: case OperationType.OP_BACKEND_TABLETS_INFO_V2: { BackendTabletsInfo backendTabletsInfo = (BackendTabletsInfo) journal.getData(); - GlobalStateMgr.getCurrentState().getLocalMetastore().replayBackendTabletsInfo(backendTabletsInfo); + GlobalStateMgr.getCurrentState().getTabletManager().replayBackendTabletsInfo(backendTabletsInfo); break; } case OperationType.OP_CREATE_ROUTINE_LOAD_JOB_V2: @@ -809,12 +806,6 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) } break; } - case OperationType.OP_MODIFY_DISTRIBUTION_TYPE: - case OperationType.OP_MODIFY_DISTRIBUTION_TYPE_V2: { - TableInfo tableInfo = (TableInfo) journal.getData(); - globalStateMgr.getLocalMetastore().replayConvertDistributionType(tableInfo); - break; - } case OperationType.OP_DYNAMIC_PARTITION: case OperationType.OP_MODIFY_IN_MEMORY: case OperationType.OP_SET_FORBIDDEN_GLOBAL_DICT: @@ -862,7 +853,7 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) } case OperationType.OP_SET_REPLICA_STATUS: { SetReplicaStatusOperationLog log = (SetReplicaStatusOperationLog) journal.getData(); - globalStateMgr.getLocalMetastore().replaySetReplicaStatus(log); + globalStateMgr.getTabletManager().replaySetReplicaStatus(log); break; } case OperationType.OP_REMOVE_ALTER_JOB_V2: { @@ -1691,10 +1682,6 @@ public void logBatchAlterJob(BatchAlterJobPersistInfo batchAlterJobV2) { logJsonObject(OperationType.OP_BATCH_ADD_ROLLUP_V2, batchAlterJobV2); } - public void logModifyDistributionType(TableInfo tableInfo) { - logJsonObject(OperationType.OP_MODIFY_DISTRIBUTION_TYPE_V2, tableInfo); - } - public void logDynamicPartition(ModifyTablePropertyOperationLog info) { logEdit(OperationType.OP_DYNAMIC_PARTITION, info); } @@ -1971,7 +1958,7 @@ public void logPipeOp(PipeOpEntry opEntry) { logEdit(OperationType.OP_PIPE, opEntry); } - private void logJsonObject(short op, Object obj) { + public void logJsonObject(short op, Object obj) { logEdit(op, out -> Text.writeString(out, GsonUtils.GSON.toJson(obj))); } diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/ModifyTablePropertyOperationLog.java b/fe/fe-core/src/main/java/com/starrocks/persist/ModifyTablePropertyOperationLog.java index 421d6ea300986..68f4f501c00ed 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/ModifyTablePropertyOperationLog.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/ModifyTablePropertyOperationLog.java @@ -20,6 +20,7 @@ import com.google.gson.annotations.SerializedName; import com.starrocks.common.io.Text; import com.starrocks.common.io.Writable; +import com.starrocks.meta.TxnMeta; import com.starrocks.persist.gson.GsonUtils; import java.io.DataInput; @@ -28,7 +29,7 @@ import java.util.HashMap; import java.util.Map; -public class ModifyTablePropertyOperationLog implements Writable { +public class ModifyTablePropertyOperationLog extends TxnMeta implements Writable { @SerializedName(value = "dbId") private long dbId; diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/ReplicaPersistInfo.java b/fe/fe-core/src/main/java/com/starrocks/persist/ReplicaPersistInfo.java index ad77d97dc8cd7..9ab5297fc7cf8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/ReplicaPersistInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/ReplicaPersistInfo.java @@ -37,12 +37,13 @@ import com.google.common.base.Objects; import com.google.gson.annotations.SerializedName; import com.starrocks.common.io.Writable; +import com.starrocks.meta.TxnMeta; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -public class ReplicaPersistInfo implements Writable { +public class ReplicaPersistInfo extends TxnMeta implements Writable { public enum ReplicaOperationType { ADD(0), diff --git a/fe/fe-core/src/main/java/com/starrocks/privilege/DbPEntryObject.java b/fe/fe-core/src/main/java/com/starrocks/privilege/DbPEntryObject.java index 1e89a795f8504..4f9d06d2a6bf7 100644 --- a/fe/fe-core/src/main/java/com/starrocks/privilege/DbPEntryObject.java +++ b/fe/fe-core/src/main/java/com/starrocks/privilege/DbPEntryObject.java @@ -137,7 +137,7 @@ public boolean isFuzzyMatching() { @Override public boolean validate(GlobalStateMgr globalStateMgr) { if (catalogId == InternalCatalog.DEFAULT_INTERNAL_CATALOG_ID) { - return globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(Long.parseLong(this.uuid)) != null; + return globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(Long.parseLong(this.uuid)) != null; } // do not validate privilege of external database return true; diff --git a/fe/fe-core/src/main/java/com/starrocks/privilege/FunctionPEntryObject.java b/fe/fe-core/src/main/java/com/starrocks/privilege/FunctionPEntryObject.java index 677911af8ef28..9ce269c117993 100644 --- a/fe/fe-core/src/main/java/com/starrocks/privilege/FunctionPEntryObject.java +++ b/fe/fe-core/src/main/java/com/starrocks/privilege/FunctionPEntryObject.java @@ -83,7 +83,7 @@ public boolean validate(GlobalStateMgr globalStateMgr) { if (databaseId == PrivilegeBuiltinConstants.GLOBAL_FUNCTION_DEFAULT_DATABASE_ID) { allFunctions = globalStateMgr.getGlobalFunctionMgr().getFunctions(); } else { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(this.databaseId); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(this.databaseId); if (db == null) { return false; } diff --git a/fe/fe-core/src/main/java/com/starrocks/privilege/PipePEntryObject.java b/fe/fe-core/src/main/java/com/starrocks/privilege/PipePEntryObject.java index dca02bca3c6f4..18ccfda45a1ed 100644 --- a/fe/fe-core/src/main/java/com/starrocks/privilege/PipePEntryObject.java +++ b/fe/fe-core/src/main/java/com/starrocks/privilege/PipePEntryObject.java @@ -111,7 +111,7 @@ public boolean isFuzzyMatching() { @Override public boolean validate(GlobalStateMgr globalStateMgr) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(Long.parseLong(this.dbUUID)); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(Long.parseLong(this.dbUUID)); if (db == null) { return false; } @@ -129,7 +129,7 @@ public List> expandObjectNames() { .map(Pipe::getDbAndName) .collect(Collectors.toList()); for (Pair dbAndName : ListUtils.emptyIfNull(dbAndNames)) { - Optional db = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(dbAndName.first); + Optional db = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(dbAndName.first); db.ifPresent(database -> objects.add( Lists.newArrayList(InternalCatalog.DEFAULT_INTERNAL_CATALOG_NAME, database.getFullName(), dbAndName.second))); @@ -206,7 +206,7 @@ public Optional getDatabase() { return Optional.empty(); } long dbId = Long.parseLong(getDbUUID()); - return GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(dbId); + return GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(dbId); } catch (NumberFormatException e) { return Optional.empty(); } diff --git a/fe/fe-core/src/main/java/com/starrocks/privilege/TablePEntryObject.java b/fe/fe-core/src/main/java/com/starrocks/privilege/TablePEntryObject.java index 0546d82b5ceab..c6fd060905b87 100644 --- a/fe/fe-core/src/main/java/com/starrocks/privilege/TablePEntryObject.java +++ b/fe/fe-core/src/main/java/com/starrocks/privilege/TablePEntryObject.java @@ -181,11 +181,11 @@ public boolean isFuzzyMatching() { @Override public boolean validate(GlobalStateMgr globalStateMgr) { if (catalogId == InternalCatalog.DEFAULT_INTERNAL_CATALOG_ID) { - Database db = globalStateMgr.getLocalMetastore().getDbIncludeRecycleBin(Long.parseLong(this.databaseUUID)); + Database db = globalStateMgr.getStarRocksMeta().getDbIncludeRecycleBin(Long.parseLong(this.databaseUUID)); if (db == null) { return false; } - return globalStateMgr.getLocalMetastore().getTableIncludeRecycleBin(db, Long.parseLong(this.tableUUID)) != null; + return globalStateMgr.getStarRocksMeta().getTableIncludeRecycleBin(db, Long.parseLong(this.tableUUID)) != null; } // do not validate privilege of external table return true; diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java b/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java index 568414f038d83..d66eddd9bb7f3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java @@ -339,7 +339,7 @@ public ShowResultSet visitCleanTemporaryTableStatement(CleanTemporaryTableStmt s @Override public ShowResultSet visitCreateMaterializedViewStmt(CreateMaterializedViewStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().createMaterializedView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().createMaterializedView(stmt); }); return null; } @@ -348,7 +348,7 @@ public ShowResultSet visitCreateMaterializedViewStmt(CreateMaterializedViewStmt public ShowResultSet visitCreateMaterializedViewStatement(CreateMaterializedViewStatement stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().createMaterializedView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().createMaterializedView(stmt); }); return null; } @@ -356,7 +356,7 @@ public ShowResultSet visitCreateMaterializedViewStatement(CreateMaterializedView @Override public ShowResultSet visitDropMaterializedViewStatement(DropMaterializedViewStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().dropMaterializedView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().dropMaterializedView(stmt); }); return null; } @@ -365,7 +365,7 @@ public ShowResultSet visitDropMaterializedViewStatement(DropMaterializedViewStmt public ShowResultSet visitAlterMaterializedViewStatement(AlterMaterializedViewStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().alterMaterializedView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().alterMaterializedView(stmt); }); return null; } @@ -376,7 +376,7 @@ public ShowResultSet visitRefreshMaterializedViewStatement(RefreshMaterializedVi List info = Lists.newArrayList(); ErrorReport.wrapWithRuntimeException(() -> { // The priority of manual refresh is higher than that of general refresh - String taskRunId = context.getGlobalStateMgr().getLocalMetastore().refreshMaterializedView(stmt); + String taskRunId = context.getGlobalStateMgr().getStarRocksMeta().refreshMaterializedView(stmt); info.add(taskRunId); }); @@ -387,7 +387,7 @@ public ShowResultSet visitRefreshMaterializedViewStatement(RefreshMaterializedVi public ShowResultSet visitCancelRefreshMaterializedViewStatement(CancelRefreshMaterializedViewStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().cancelRefreshMaterializedView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().cancelRefreshMaterializedView(stmt); }); return null; } @@ -403,7 +403,7 @@ public ShowResultSet visitAlterTableStatement(AlterTableStmt stmt, ConnectContex @Override public ShowResultSet visitAlterViewStatement(AlterViewStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().alterView(stmt); + context.getGlobalStateMgr().getStarRocksMeta().alterView(stmt); }); return null; } @@ -411,7 +411,7 @@ public ShowResultSet visitAlterViewStatement(AlterViewStmt stmt, ConnectContext @Override public ShowResultSet visitCancelAlterTableStatement(CancelAlterTableStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().cancelAlter(stmt); + context.getGlobalStateMgr().getAlterJobMgr().cancelAlter(stmt, "user cancelled"); }); return null; } @@ -604,7 +604,7 @@ public ShowResultSet visitCancelAlterSystemStatement(CancelAlterSystemStmt stmt, @Override public ShowResultSet visitAlterDatabaseQuotaStatement(AlterDatabaseQuotaStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().alterDatabaseQuota(stmt); + context.getGlobalStateMgr().getStarRocksMeta().alterDatabaseQuota(stmt); }); return null; } @@ -613,7 +613,7 @@ public ShowResultSet visitAlterDatabaseQuotaStatement(AlterDatabaseQuotaStmt stm public ShowResultSet visitAlterDatabaseRenameStatement(AlterDatabaseRenameStatement stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().renameDatabase(stmt); + context.getGlobalStateMgr().getStarRocksMeta().renameDatabase(stmt); }); return null; } @@ -621,7 +621,7 @@ public ShowResultSet visitAlterDatabaseRenameStatement(AlterDatabaseRenameStatem @Override public ShowResultSet visitRecoverDbStatement(RecoverDbStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().recoverDatabase(stmt); + context.getGlobalStateMgr().getStarRocksMeta().recoverDatabase(stmt); }); return null; } @@ -629,7 +629,7 @@ public ShowResultSet visitRecoverDbStatement(RecoverDbStmt stmt, ConnectContext @Override public ShowResultSet visitRecoverTableStatement(RecoverTableStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().recoverTable(stmt); + context.getGlobalStateMgr().getStarRocksMeta().recoverTable(stmt); }); return null; } @@ -637,7 +637,7 @@ public ShowResultSet visitRecoverTableStatement(RecoverTableStmt stmt, ConnectCo @Override public ShowResultSet visitRecoverPartitionStatement(RecoverPartitionStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().recoverPartition(stmt); + context.getGlobalStateMgr().getStarRocksMeta().recoverPartition(stmt); }); return null; } @@ -700,7 +700,7 @@ public ShowResultSet visitSyncStatement(SyncStmt stmt, ConnectContext context) { @Override public ShowResultSet visitTruncateTableStatement(TruncateTableStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().truncateTable(stmt, context); + context.getGlobalStateMgr().getStarRocksMeta().truncateTable(stmt, context); }); return null; } @@ -784,7 +784,7 @@ public ShowResultSet visitUninstallPluginStatement(UninstallPluginStmt stmt, Con @Override public ShowResultSet visitAdminCheckTabletsStatement(AdminCheckTabletsStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().checkTablets(stmt); + context.getGlobalStateMgr().getTabletManager().checkTablets(stmt); }); return null; } @@ -792,7 +792,7 @@ public ShowResultSet visitAdminCheckTabletsStatement(AdminCheckTabletsStmt stmt, @Override public ShowResultSet visitAdminSetPartitionVersionStmt(AdminSetPartitionVersionStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() - -> context.getGlobalStateMgr().getLocalMetastore().setPartitionVersion(stmt)); + -> context.getGlobalStateMgr().getStarRocksMeta().setPartitionVersion(stmt)); return null; } @@ -800,7 +800,7 @@ public ShowResultSet visitAdminSetPartitionVersionStmt(AdminSetPartitionVersionS public ShowResultSet visitAdminSetReplicaStatusStatement(AdminSetReplicaStatusStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { - context.getGlobalStateMgr().getLocalMetastore().setReplicaStatus(stmt); + context.getGlobalStateMgr().getTabletManager().setReplicaStatus(stmt); }); return null; } diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/ShowExecutor.java b/fe/fe-core/src/main/java/com/starrocks/qe/ShowExecutor.java index 4e7fb2ebc1c46..22fc04484b86c 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/ShowExecutor.java @@ -1617,7 +1617,7 @@ public ShowResultSet visitShowTabletStatement(ShowTabletStmt statement, ConnectC break; } - List replicas = tablet.getImmutableReplicas(); + List replicas = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllReplicas(tablet); for (Replica replica : replicas) { Replica tmp = invertedIndex.getReplica(tabletId, replica.getBackendId()); if (tmp == null) { @@ -2503,7 +2503,7 @@ public ShowResultSet visitDescStorageVolumeStatement(DescStorageVolumeStmt state public ShowResultSet visitShowPipeStatement(ShowPipeStmt statement, ConnectContext context) { List> rows = Lists.newArrayList(); String dbName = statement.getDbName(); - long dbId = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(dbName) + long dbId = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(dbName) .map(Database::getId) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_BAD_DB_ERROR, dbName)); PipeManager pipeManager = GlobalStateMgr.getCurrentState().getPipeManager(); diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/MVActiveChecker.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/MVActiveChecker.java index 1bcba036ca4c3..1dc0e75ad33aa 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/MVActiveChecker.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/MVActiveChecker.java @@ -125,7 +125,8 @@ public static void tryToActivate(MaterializedView mv, boolean checkGracePeriod) } long dbId = mv.getDbId(); - Optional dbName = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(dbId).map(Database::getFullName); + Optional dbName = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .mayGetDb(dbId).map(Database::getFullName); if (!dbName.isPresent()) { LOG.warn("[MVActiveChecker] cannot activate MV {} since database {} not found", mv.getName(), dbId); return; diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/history/TableKeeper.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/history/TableKeeper.java index 9bd41bf2f0ef6..8c70654f73ccf 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/history/TableKeeper.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/history/TableKeeper.java @@ -102,7 +102,7 @@ public void createTable() throws UserException { public void correctTable() { int numBackends = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getTotalBackendNumber(); int replica = GlobalStateMgr.getCurrentState() - .getLocalMetastore().mayGetTable(databaseName, tableName) + .getStarRocksMeta().mayGetTable(databaseName, tableName) .map(tbl -> ((OlapTable) tbl).getPartitionInfo().getMinReplicationNum()) .orElse((short) 1); if (numBackends < tableReplicas) { @@ -142,7 +142,7 @@ public void changeTTL() { private Optional mayGetTable() { return GlobalStateMgr.getCurrentState() - .getLocalMetastore().mayGetTable(databaseName, tableName) + .getStarRocksMeta().mayGetTable(databaseName, tableName) .flatMap(x -> Optional.of((OlapTable) x)); } diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/IMTCreator.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/IMTCreator.java index ec8c46d20724c..ad647ee0ed699 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/IMTCreator.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/IMTCreator.java @@ -120,7 +120,7 @@ public static void createIMT(CreateMaterializedViewStatement stmt, MaterializedV for (CreateTableStmt create : createTables) { LOG.info("creating IMT {} for MV {}", create.getTableName(), view.getName()); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(create); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(create); } catch (DdlException e) { // TODO(murphy) cleanup created IMT, or make it atomic LOG.warn("create IMT {} failed due to ", create.getTableName(), e); diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshListPartitioner.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshListPartitioner.java index fdd5de0b303d1..450be15034b79 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshListPartitioner.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshListPartitioner.java @@ -323,7 +323,7 @@ private void addListPartitions(Database database, MaterializedView materializedV AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(materializedView); analyzer.analyze(mvContext.getCtx(), addPartitionClause); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().addPartitions( + GlobalStateMgr.getCurrentState().getStarRocksMeta().addPartitions( mvContext.getCtx(), database, materializedView.getName(), addPartitionClause); } catch (Exception e) { throw new DmlException("add list partition failed: %s, db: %s, table: %s", e, e.getMessage(), diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshPartitioner.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshPartitioner.java index 0909f335d6b81..4dac4d147e3cc 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshPartitioner.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshPartitioner.java @@ -272,7 +272,7 @@ protected void dropPartition(Database db, MaterializedView materializedView, Str AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(materializedView); analyzer.analyze(new ConnectContext(), dropPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(db, materializedView, dropPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropPartition(db, materializedView, dropPartitionClause); } catch (Exception e) { throw new DmlException("Expression add partition failed: %s, db: %s, table: %s", e, e.getMessage(), db.getFullName(), materializedView.getName()); diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshRangePartitioner.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshRangePartitioner.java index e7e788fd2c0c5..2661cb6814700 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshRangePartitioner.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/mv/MVPCTRefreshRangePartitioner.java @@ -465,7 +465,7 @@ private void addRangePartitions(Database database, MaterializedView materialized AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(materializedView); analyzer.analyze(mvContext.getCtx(), alterPartition); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().addPartitions(mvContext.getCtx(), + GlobalStateMgr.getCurrentState().getStarRocksMeta().addPartitions(mvContext.getCtx(), database, materializedView.getName(), alterPartition); } catch (Exception e) { throw new DmlException("Expression add partition failed: %s, db: %s, table: %s", e, e.getMessage(), diff --git a/fe/fe-core/src/main/java/com/starrocks/server/ElasticSearchTableFactory.java b/fe/fe-core/src/main/java/com/starrocks/server/ElasticSearchTableFactory.java index 0f383d5fdb5e7..fba6f310f9199 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/ElasticSearchTableFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/ElasticSearchTableFactory.java @@ -55,7 +55,7 @@ public Table createTable(LocalMetastore metastore, Database database, CreateTabl .collect(Collectors.toList()); // metastore is null when external table if (null != metastore) { - metastore.validateColumns(baseSchema); + GlobalStateMgr.getCurrentState().getStarRocksMeta().validateColumns(baseSchema); } // create partition info @@ -65,7 +65,7 @@ public Table createTable(LocalMetastore metastore, Database database, CreateTabl if (partitionDesc != null) { partitionInfo = partitionDesc.toPartitionInfo(baseSchema, partitionNameToId, false); } else if (null != metastore) { - long partitionId = metastore.getNextId(); + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); // use table name as single partition name partitionNameToId.put(tableName, partitionId); partitionInfo = new SinglePartitionInfo(); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java index 8074bfc836ee0..413c1c21effff 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java @@ -127,6 +127,7 @@ import com.starrocks.journal.JournalInconsistentException; import com.starrocks.journal.JournalTask; import com.starrocks.journal.JournalWriter; +import com.starrocks.journal.bdbje.BDBJEJournal; import com.starrocks.journal.bdbje.Timestamp; import com.starrocks.lake.ShardManager; import com.starrocks.lake.StarMgrMetaSyncer; @@ -157,8 +158,11 @@ import com.starrocks.memory.MemoryUsageTracker; import com.starrocks.memory.ProcProfileCollector; import com.starrocks.meta.MetaContext; +import com.starrocks.meta.MetadataHandler; +import com.starrocks.meta.StarRocksMeta; +import com.starrocks.meta.TabletManager; +import com.starrocks.meta.TabletMetastore; import com.starrocks.metric.MetricRepo; -import com.starrocks.persist.BackendIdsUpdateInfo; import com.starrocks.persist.EditLog; import com.starrocks.persist.ImageFormatVersion; import com.starrocks.persist.ImageHeader; @@ -346,6 +350,8 @@ public class GlobalStateMgr { private EditLog editLog; private Journal journal; + private MetadataHandler metadataHandler; + // For checkpoint and observer memory replayed marker private final AtomicLong replayedJournalId; @@ -358,6 +364,9 @@ public class GlobalStateMgr { private final JournalObservable journalObservable; private final TabletInvertedIndex tabletInvertedIndex; + private final TabletMetastore tabletMetastore; + private final TabletManager tabletManager; + private ColocateTableIndex colocateTableIndex; private final CatalogRecycleBin recycleBin; @@ -439,6 +448,7 @@ public class GlobalStateMgr { private final InsertOverwriteJobMgr insertOverwriteJobMgr; private final LocalMetastore localMetastore; + private final StarRocksMeta starRocksMeta; private final GlobalFunctionMgr globalFunctionMgr; @Deprecated @@ -534,10 +544,17 @@ public TabletInvertedIndex getTabletInvertedIndex() { return this.tabletInvertedIndex; } + public TabletMetastore getTabletMetastore() { + return tabletMetastore; + } + + public TabletManager getTabletManager() { + return tabletManager; + } + // only for test public void setColocateTableIndex(ColocateTableIndex colocateTableIndex) { this.colocateTableIndex = colocateTableIndex; - localMetastore.setColocateTableIndex(colocateTableIndex); } public ColocateTableIndex getColocateTableIndex() { @@ -564,6 +581,10 @@ public LocalMetastore getLocalMetastore() { return localMetastore; } + public StarRocksMeta getStarRocksMeta() { + return starRocksMeta; + } + public TemporaryTableMgr getTemporaryTableMgr() { return temporaryTableMgr; } @@ -644,6 +665,9 @@ private GlobalStateMgr(boolean isCkptGlobalState, NodeMgr nodeMgr) { this.journalObservable = new JournalObservable(); this.tabletInvertedIndex = new TabletInvertedIndex(); + this.tabletMetastore = new TabletMetastore(); + this.tabletManager = new TabletManager(); + this.colocateTableIndex = new ColocateTableIndex(); this.recycleBin = new CatalogRecycleBin(); this.functionSet = new FunctionSet(); @@ -704,12 +728,13 @@ private GlobalStateMgr(boolean isCkptGlobalState, NodeMgr nodeMgr) { this.pluginMgr = new PluginMgr(); this.auditEventProcessor = new AuditEventProcessor(this.pluginMgr); this.analyzeMgr = new AnalyzeMgr(); - this.localMetastore = new LocalMetastore(this, recycleBin, colocateTableIndex); + this.localMetastore = new LocalMetastore(this, recycleBin); + this.starRocksMeta = new StarRocksMeta(); this.temporaryTableMgr = new TemporaryTableMgr(); this.warehouseMgr = new WarehouseManager(); this.connectorMgr = new ConnectorMgr(); this.connectorTblMetaInfoMgr = new ConnectorTblMetaInfoMgr(); - this.metadataMgr = new MetadataMgr(localMetastore, temporaryTableMgr, connectorMgr, connectorTblMetaInfoMgr); + this.metadataMgr = new MetadataMgr(starRocksMeta, temporaryTableMgr, connectorMgr, connectorTblMetaInfoMgr); this.catalogMgr = new CatalogMgr(connectorMgr); this.connectorTableTriggerAnalyzeMgr = new ConnectorTableTriggerAnalyzeMgr(); @@ -1150,6 +1175,8 @@ protected void initJournal() throws JournalException, InterruptedException { journalWriter = new JournalWriter(journal, journalQueue); editLog = new EditLog(journalQueue); + + metadataHandler = new MetadataHandler(((BDBJEJournal) journal).getBdbEnvironment()); } // wait until FE is ready. @@ -2062,6 +2089,10 @@ public Journal getJournal() { return journal; } + public MetadataHandler getMetadataHandler() { + return metadataHandler; + } + // Get the next available, lock-free because nextId is atomic. public long getNextId() { return idGenerator.getNextId(); @@ -2429,10 +2460,6 @@ public void initDefaultWarehouse() { isDefaultWarehouseCreated = true; } - public void replayUpdateClusterAndBackends(BackendIdsUpdateInfo info) { - localMetastore.replayUpdateClusterAndBackends(info); - } - public String dumpImage() { LOG.info("begin to dump meta data"); String dumpFilePath; diff --git a/fe/fe-core/src/main/java/com/starrocks/server/JDBCTableFactory.java b/fe/fe-core/src/main/java/com/starrocks/server/JDBCTableFactory.java index 9120b4f03e10c..f89488651ae66 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/JDBCTableFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/JDBCTableFactory.java @@ -37,7 +37,7 @@ public Table createTable(LocalMetastore metastore, Database database, CreateTabl String tableName = stmt.getTableName(); List columns = stmt.getColumns(); Map properties = stmt.getProperties(); - long tableId = metastore.getNextId(); + long tableId = GlobalStateMgr.getCurrentState().getNextId(); return new JDBCTable(tableId, tableName, columns, properties); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java index e16c3488f982a..e8448693173b5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java @@ -35,137 +35,68 @@ package com.starrocks.server; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; -import com.google.common.collect.Range; import com.google.common.collect.Sets; -import com.staros.proto.FilePathInfo; -import com.starrocks.alter.AlterJobExecutor; -import com.starrocks.alter.AlterMVJobExecutor; -import com.starrocks.alter.MaterializedViewHandler; -import com.starrocks.analysis.Expr; -import com.starrocks.analysis.FunctionCallExpr; -import com.starrocks.analysis.HintNode; -import com.starrocks.analysis.IntLiteral; -import com.starrocks.analysis.SetVarHint; -import com.starrocks.analysis.SlotRef; -import com.starrocks.analysis.StringLiteral; -import com.starrocks.analysis.TableName; -import com.starrocks.analysis.TableRef; -import com.starrocks.analysis.UserVariableHint; -import com.starrocks.binlog.BinlogConfig; import com.starrocks.catalog.CatalogRecycleBin; -import com.starrocks.catalog.CatalogUtils; -import com.starrocks.catalog.ColocateGroupSchema; -import com.starrocks.catalog.ColocateTableIndex; import com.starrocks.catalog.Column; -import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; -import com.starrocks.catalog.DistributionInfo; -import com.starrocks.catalog.FunctionSet; -import com.starrocks.catalog.HashDistributionInfo; import com.starrocks.catalog.HiveTable; -import com.starrocks.catalog.Index; -import com.starrocks.catalog.KeysType; import com.starrocks.catalog.ListPartitionInfo; import com.starrocks.catalog.LocalTablet; import com.starrocks.catalog.MaterializedIndex; -import com.starrocks.catalog.MaterializedIndex.IndexExtState; -import com.starrocks.catalog.MaterializedIndexMeta; -import com.starrocks.catalog.MaterializedView; -import com.starrocks.catalog.MvId; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; import com.starrocks.catalog.PartitionInfo; -import com.starrocks.catalog.PartitionKey; import com.starrocks.catalog.PartitionType; import com.starrocks.catalog.PhysicalPartition; -import com.starrocks.catalog.PhysicalPartitionImpl; -import com.starrocks.catalog.PrimitiveType; -import com.starrocks.catalog.RandomDistributionInfo; import com.starrocks.catalog.RangePartitionInfo; import com.starrocks.catalog.Replica; -import com.starrocks.catalog.SinglePartitionInfo; import com.starrocks.catalog.Table; import com.starrocks.catalog.TableProperty; import com.starrocks.catalog.Tablet; import com.starrocks.catalog.TabletInvertedIndex; import com.starrocks.catalog.TabletMeta; -import com.starrocks.catalog.View; import com.starrocks.catalog.system.information.InfoSchemaDb; import com.starrocks.catalog.system.sys.SysDb; import com.starrocks.cluster.ClusterNamespace; -import com.starrocks.common.AlreadyExistsException; import com.starrocks.common.AnalysisException; -import com.starrocks.common.Config; import com.starrocks.common.DdlException; -import com.starrocks.common.ErrorCode; -import com.starrocks.common.ErrorReport; -import com.starrocks.common.ErrorReportException; -import com.starrocks.common.InvalidOlapTableStateException; import com.starrocks.common.MaterializedViewExceptions; -import com.starrocks.common.MetaNotFoundException; import com.starrocks.common.Pair; -import com.starrocks.common.UserException; import com.starrocks.common.util.DynamicPartitionUtil; import com.starrocks.common.util.PropertyAnalyzer; -import com.starrocks.common.util.TimeUtils; import com.starrocks.common.util.UUIDUtil; -import com.starrocks.common.util.Util; -import com.starrocks.common.util.concurrent.CountingLatch; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; -import com.starrocks.connector.ConnectorMetadata; import com.starrocks.connector.exception.StarRocksConnectorException; -import com.starrocks.lake.DataCacheInfo; -import com.starrocks.lake.LakeMaterializedView; -import com.starrocks.lake.LakeTable; -import com.starrocks.lake.LakeTablet; -import com.starrocks.lake.StorageInfo; -import com.starrocks.load.pipe.PipeManager; import com.starrocks.memory.MemoryTrackable; +import com.starrocks.meta.LocalMetastoreInterface; import com.starrocks.mv.MVMetaVersionRepairer; import com.starrocks.mv.MVRepairHandler; -import com.starrocks.mv.analyzer.MVPartitionExprResolver; import com.starrocks.persist.AddPartitionsInfoV2; import com.starrocks.persist.AddSubPartitionsInfoV2; import com.starrocks.persist.AutoIncrementInfo; -import com.starrocks.persist.BackendIdsUpdateInfo; -import com.starrocks.persist.BackendTabletsInfo; -import com.starrocks.persist.BatchDeleteReplicaInfo; -import com.starrocks.persist.ColocatePersistInfo; import com.starrocks.persist.ColumnRenameInfo; import com.starrocks.persist.CreateDbInfo; import com.starrocks.persist.CreateTableInfo; import com.starrocks.persist.DatabaseInfo; -import com.starrocks.persist.DisablePartitionRecoveryInfo; -import com.starrocks.persist.DisableTableRecoveryInfo; import com.starrocks.persist.DropDbInfo; +import com.starrocks.persist.DropInfo; import com.starrocks.persist.DropPartitionInfo; import com.starrocks.persist.DropPartitionsInfo; -import com.starrocks.persist.EditLog; import com.starrocks.persist.ImageWriter; import com.starrocks.persist.ListPartitionPersistInfo; -import com.starrocks.persist.ModifyPartitionInfo; import com.starrocks.persist.ModifyTableColumnOperationLog; import com.starrocks.persist.ModifyTablePropertyOperationLog; -import com.starrocks.persist.MultiEraseTableInfo; import com.starrocks.persist.OperationType; -import com.starrocks.persist.PartitionPersistInfo; import com.starrocks.persist.PartitionPersistInfoV2; import com.starrocks.persist.PartitionVersionRecoveryInfo; -import com.starrocks.persist.PartitionVersionRecoveryInfo.PartitionVersion; import com.starrocks.persist.PhysicalPartitionPersistInfoV2; import com.starrocks.persist.RangePartitionPersistInfo; import com.starrocks.persist.RecoverInfo; import com.starrocks.persist.ReplacePartitionOperationLog; -import com.starrocks.persist.ReplicaPersistInfo; -import com.starrocks.persist.SetReplicaStatusOperationLog; import com.starrocks.persist.TableInfo; import com.starrocks.persist.TruncateTableInfo; import com.starrocks.persist.metablock.SRMetaBlockEOFException; @@ -173,101 +104,32 @@ import com.starrocks.persist.metablock.SRMetaBlockID; import com.starrocks.persist.metablock.SRMetaBlockReader; import com.starrocks.persist.metablock.SRMetaBlockWriter; -import com.starrocks.privilege.AccessDeniedException; -import com.starrocks.privilege.ObjectType; -import com.starrocks.privilege.PrivilegeType; -import com.starrocks.qe.ConnectContext; -import com.starrocks.qe.SessionVariable; -import com.starrocks.qe.VariableMgr; -import com.starrocks.scheduler.Constants; -import com.starrocks.scheduler.ExecuteOption; -import com.starrocks.scheduler.Task; -import com.starrocks.scheduler.TaskBuilder; -import com.starrocks.scheduler.TaskManager; -import com.starrocks.scheduler.TaskRun; -import com.starrocks.sql.analyzer.AnalyzerUtils; -import com.starrocks.sql.analyzer.Authorizer; -import com.starrocks.sql.ast.AddPartitionClause; -import com.starrocks.sql.ast.AdminCheckTabletsStmt; -import com.starrocks.sql.ast.AdminSetPartitionVersionStmt; -import com.starrocks.sql.ast.AdminSetReplicaStatusStmt; import com.starrocks.sql.ast.AlterDatabaseQuotaStmt; -import com.starrocks.sql.ast.AlterDatabaseRenameStatement; -import com.starrocks.sql.ast.AlterMaterializedViewStmt; -import com.starrocks.sql.ast.AlterTableCommentClause; -import com.starrocks.sql.ast.AlterTableStmt; -import com.starrocks.sql.ast.AlterViewStmt; -import com.starrocks.sql.ast.AsyncRefreshSchemeDesc; -import com.starrocks.sql.ast.CancelAlterTableStmt; -import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; -import com.starrocks.sql.ast.ColumnRenameClause; -import com.starrocks.sql.ast.CreateMaterializedViewStatement; -import com.starrocks.sql.ast.CreateMaterializedViewStmt; -import com.starrocks.sql.ast.CreateTableLikeStmt; -import com.starrocks.sql.ast.CreateTableStmt; -import com.starrocks.sql.ast.CreateTemporaryTableStmt; -import com.starrocks.sql.ast.CreateViewStmt; -import com.starrocks.sql.ast.DistributionDesc; -import com.starrocks.sql.ast.DropMaterializedViewStmt; -import com.starrocks.sql.ast.DropPartitionClause; -import com.starrocks.sql.ast.DropTableStmt; -import com.starrocks.sql.ast.ExpressionPartitionDesc; -import com.starrocks.sql.ast.IntervalLiteral; import com.starrocks.sql.ast.PartitionDesc; -import com.starrocks.sql.ast.PartitionRangeDesc; -import com.starrocks.sql.ast.PartitionRenameClause; -import com.starrocks.sql.ast.RecoverDbStmt; -import com.starrocks.sql.ast.RecoverPartitionStmt; -import com.starrocks.sql.ast.RecoverTableStmt; -import com.starrocks.sql.ast.RefreshMaterializedViewStatement; -import com.starrocks.sql.ast.RefreshSchemeClause; -import com.starrocks.sql.ast.ReplacePartitionClause; -import com.starrocks.sql.ast.RollupRenameClause; -import com.starrocks.sql.ast.ShowAlterStmt; import com.starrocks.sql.ast.SingleRangePartitionDesc; -import com.starrocks.sql.ast.SystemVariable; -import com.starrocks.sql.ast.TableRenameClause; -import com.starrocks.sql.ast.TruncateTableStmt; -import com.starrocks.sql.common.MetaUtils; -import com.starrocks.sql.common.SyncPartitionUtils; -import com.starrocks.sql.optimizer.Utils; import com.starrocks.sql.optimizer.statistics.IDictManager; -import com.starrocks.system.Backend; -import com.starrocks.system.SystemInfoService; -import com.starrocks.task.TabletTaskExecutor; -import com.starrocks.thrift.TGetTasksParams; import com.starrocks.thrift.TStorageMedium; -import com.starrocks.thrift.TStorageType; -import com.starrocks.thrift.TTabletMetaType; -import com.starrocks.thrift.TTabletType; -import com.starrocks.warehouse.Warehouse; -import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.io.IOException; -import java.time.LocalDateTime; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import javax.validation.constraints.NotNull; +import static com.starrocks.meta.StarRocksMeta.inactiveRelatedMaterializedView; import static com.starrocks.server.GlobalStateMgr.NEXT_ID_INIT_VALUE; import static com.starrocks.server.GlobalStateMgr.isCheckpointThread; -public class LocalMetastore implements ConnectorMetadata, MVRepairHandler, MemoryTrackable { +public class LocalMetastore implements MVRepairHandler, MemoryTrackable, LocalMetastoreInterface { private static final Logger LOG = LogManager.getLogger(LocalMetastore.class); private final ConcurrentHashMap idToDb = new ConcurrentHashMap<>(); @@ -276,20 +138,10 @@ public class LocalMetastore implements ConnectorMetadata, MVRepairHandler, Memor private final GlobalStateMgr stateMgr; private final CatalogRecycleBin recycleBin; - private ColocateTableIndex colocateTableIndex; - /** - * Concurrent colocate table creation process have dependency on each other - * (even in different databases), but we do not want to affect the performance - * of non-colocate table creation, so here we use a separate latch to - * synchronize only the creation of colocate tables. - */ - private final CountingLatch colocateTableCreateSyncer = new CountingLatch(0); - - public LocalMetastore(GlobalStateMgr globalStateMgr, CatalogRecycleBin recycleBin, - ColocateTableIndex colocateTableIndex) { + + public LocalMetastore(GlobalStateMgr globalStateMgr, CatalogRecycleBin recycleBin) { this.stateMgr = globalStateMgr; this.recycleBin = recycleBin; - this.colocateTableIndex = colocateTableIndex; // put built-in database into local metastore InfoSchemaDb infoSchemaDb = new InfoSchemaDb(); @@ -305,90 +157,10 @@ public LocalMetastore(GlobalStateMgr globalStateMgr, CatalogRecycleBin recycleBi fullNameToDb.put(starRocksDb.getFullName(), starRocksDb); } - boolean tryLock(boolean mustLock) { - return stateMgr.tryLock(mustLock); - } - - void unlock() { - stateMgr.unlock(); - } - - long getNextId() { - return stateMgr.getNextId(); - } - - GlobalStateMgr getStateMgr() { - return stateMgr; - } - - public void recreateTabletInvertIndex() { - if (isCheckpointThread()) { - return; - } - - // create inverted index - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (Database db : this.fullNameToDb.values()) { - long dbId = db.getId(); - for (Table table : db.getTables()) { - if (!table.isNativeTableOrMaterializedView()) { - continue; - } - - OlapTable olapTable = (OlapTable) table; - long tableId = olapTable.getId(); - for (PhysicalPartition partition : olapTable.getAllPhysicalPartitions()) { - long physicalPartitionId = partition.getId(); - TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( - partition.getParentId()).getStorageMedium(); - for (MaterializedIndex index : partition - .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - long indexId = index.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partition.getParentId(), physicalPartitionId, - indexId, schemaHash, medium, table.isCloudNativeTableOrMaterializedView()); - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - if (table.isOlapTableOrMaterializedView()) { - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } // end for indices - } // end for partitions - } // end for tables - } // end for dbs - } - @Override - public void createDb(String dbName, Map properties) throws DdlException, AlreadyExistsException { - long id = 0L; - if (!tryLock(false)) { - throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); - } - try { - if (fullNameToDb.containsKey(dbName)) { - throw new AlreadyExistsException("Database Already Exists"); - } else { - id = getNextId(); - Database db = new Database(id, dbName); - String volume = StorageVolumeMgr.DEFAULT; - if (properties != null && properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_VOLUME)) { - volume = properties.remove(PropertyAnalyzer.PROPERTIES_STORAGE_VOLUME); - } - if (!GlobalStateMgr.getCurrentState().getStorageVolumeMgr().bindDbToStorageVolume(volume, id)) { - throw new DdlException(String.format("Storage volume %s not exists", volume)); - } - unprotectCreateDb(db); - String storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr().getStorageVolumeIdOfDb(id); - GlobalStateMgr.getCurrentState().getEditLog().logCreateDb(db, storageVolumeId); - } - } finally { - unlock(); - } - LOG.info("createDb dbName = " + dbName + ", id = " + id); + public void createDb(Database db, String storageVolumeId) { + unprotectCreateDb(db); + GlobalStateMgr.getCurrentState().getEditLog().logCreateDb(db, storageVolumeId); } // For replay edit log, needn't lock metadata @@ -402,104 +174,35 @@ public void unprotectCreateDb(Database db) { stateMgr.getGlobalTransactionMgr().addDatabaseTransactionMgr(db.getId()); } - public ConcurrentHashMap getIdToDb() { - return idToDb; - } - - public void replayCreateDb(Database db) { - tryLock(true); - try { - unprotectCreateDb(db); - LOG.info("finish replay create db, name: {}, id: {}", db.getOriginName(), db.getId()); - } finally { - unlock(); - } - } - public void replayCreateDb(CreateDbInfo createDbInfo) { - tryLock(true); + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); + LocalMetastore localMetastore = globalStateMgr.getLocalMetastore(); + + globalStateMgr.tryLock(true); try { Database db = new Database(createDbInfo.getId(), createDbInfo.getDbName()); - unprotectCreateDb(db); + localMetastore.unprotectCreateDb(db); + GlobalStateMgr.getCurrentState().getGlobalTransactionMgr().addDatabaseTransactionMgr(db.getId()); + // If user upgrades from 3.0, the storage volume id will be null if (createDbInfo.getStorageVolumeId() != null) { - stateMgr.getStorageVolumeMgr().replayBindDbToStorageVolume(createDbInfo.getStorageVolumeId(), db.getId()); + globalStateMgr.getStorageVolumeMgr().replayBindDbToStorageVolume(createDbInfo.getStorageVolumeId(), db.getId()); } LOG.info("finish replay create db, name: {}, id: {}", db.getOriginName(), db.getId()); } finally { - unlock(); + globalStateMgr.unlock(); } } @Override - public void dropDb(String dbName, boolean isForceDrop) throws DdlException, MetaNotFoundException { - // 1. check if database exists - Database db; - if (!tryLock(false)) { - throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); - } - try { - if (!fullNameToDb.containsKey(dbName)) { - throw new MetaNotFoundException("Database not found"); - } - db = this.fullNameToDb.get(dbName); - if (!isForceDrop && !db.getTemporaryTables().isEmpty()) { - throw new DdlException("The database [" + dbName + "] " + - "cannot be dropped because there are still some temporary tables in it. " + - "If you want to forcibly drop, please use \"DROP DATABASE FORCE.\""); - } - } finally { - unlock(); - } - - // 2. drop tables in db - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - if (!db.isExist()) { - throw new MetaNotFoundException("Database '" + dbName + "' not found"); - } - if (!isForceDrop && stateMgr.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\"."); - } - - // save table names for recycling - Set tableNames = new HashSet<>(db.getTableNamesViewWithLock()); - unprotectDropDb(db, isForceDrop, false); - if (!isForceDrop) { - recycleBin.recycleDatabase(db, tableNames); - } else { - stateMgr.getLocalMetastore().onEraseDatabase(db.getId()); - } - db.setExist(false); + public void dropDb(Database db, boolean isForceDrop) { + // 3. remove db from globalStateMgr + idToDb.remove(db.getId()); + fullNameToDb.remove(db.getFullName()); + unprotectDropDb(db, isForceDrop, false); - // 3. remove db from globalStateMgr - idToDb.remove(db.getId()); - fullNameToDb.remove(db.getFullName()); - - // 4. drop mv task - TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); - TGetTasksParams tasksParams = new TGetTasksParams(); - tasksParams.setDb(dbName); - List dropTaskIdList = taskManager.filterTasks(tasksParams) - .stream().map(Task::getId).collect(Collectors.toList()); - taskManager.dropTasks(dropTaskIdList, false); - - DropDbInfo info = new DropDbInfo(db.getFullName(), isForceDrop); - GlobalStateMgr.getCurrentState().getEditLog().logDropDb(info); - - // 5. Drop Pipes - PipeManager pipeManager = GlobalStateMgr.getCurrentState().getPipeManager(); - pipeManager.dropPipesOfDb(dbName, db.getId()); - - LOG.info("finish drop database[{}], id: {}, is force : {}", dbName, db.getId(), isForceDrop); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } + DropDbInfo info = new DropDbInfo(db.getFullName(), isForceDrop); + GlobalStateMgr.getCurrentState().getEditLog().logDropDb(info); } @NotNull @@ -517,8 +220,8 @@ public void unprotectDropDb(Database db, boolean isForeDrop, boolean isReplay) { } } - public void replayDropDb(String dbName, boolean isForceDrop) throws DdlException { - tryLock(true); + public void replayDropDb(String dbName, boolean isForceDrop) { + GlobalStateMgr.getCurrentState().tryLock(true); try { Database db = fullNameToDb.get(dbName); Locker locker = new Locker(); @@ -526,11 +229,7 @@ public void replayDropDb(String dbName, boolean isForceDrop) throws DdlException try { Set tableNames = new HashSet(db.getTableNamesViewWithLock()); unprotectDropDb(db, isForceDrop, true); - if (!isForceDrop) { - recycleBin.recycleDatabase(db, tableNames); - } else { - stateMgr.getLocalMetastore().onEraseDatabase(db.getId()); - } + recycleBin.recycleDatabase(db, tableNames, isForceDrop); db.setExist(false); } finally { locker.unLockDatabase(db.getId(), LockType.WRITE); @@ -541,144 +240,37 @@ public void replayDropDb(String dbName, boolean isForceDrop) throws DdlException LOG.info("finish replay drop db, name: {}, id: {}", dbName, db.getId()); } finally { - unlock(); - } - } - - public void recoverDatabase(RecoverDbStmt recoverStmt) throws DdlException { - // check is new db with same name already exist - if (getDb(recoverStmt.getDbName()) != null) { - throw new DdlException("Database[" + recoverStmt.getDbName() + "] already exist."); - } - - Database db = recycleBin.recoverDatabase(recoverStmt.getDbName()); - - // add db to globalStateMgr - if (!tryLock(false)) { - throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); - } - try { - if (fullNameToDb.containsKey(db.getFullName())) { - throw new DdlException("Database[" + db.getOriginName() + "] already exist."); - // it's ok that we do not put db back to CatalogRecycleBin - // cause this db cannot recover anymore - } - - fullNameToDb.put(db.getFullName(), db); - idToDb.put(db.getId(), db); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - db.setExist(true); - locker.unLockDatabase(db.getId(), LockType.WRITE); - - List materializedViews = db.getMaterializedViews(); - TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); - for (MaterializedView materializedView : materializedViews) { - MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); - if (refreshType != MaterializedView.RefreshType.SYNC) { - Task task = TaskBuilder.buildMvTask(materializedView, db.getFullName()); - TaskBuilder.updateTaskInfo(task, materializedView); - taskManager.createTask(task, false); - } - } - - // log - RecoverInfo recoverInfo = new RecoverInfo(db.getId(), -1L, -1L); - GlobalStateMgr.getCurrentState().getEditLog().logRecoverDb(recoverInfo); - } finally { - unlock(); + GlobalStateMgr.getCurrentState().unlock(); } - - LOG.info("finish recover database, name: {}, id: {}", recoverStmt.getDbName(), db.getId()); } - 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(); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - Table table = getTable(db.getFullName(), tableName); - if (table != null) { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } - - if (!recycleBin.recoverTable(db, tableName)) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - Table recoverTable = getTable(db.getFullName(), tableName); - if (recoverTable instanceof OlapTable) { - DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), (OlapTable) recoverTable); - } - - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } + @Override + public void recoverDatabase(Database db) { + unprotectCreateDb(db); + // log + RecoverInfo recoverInfo = new RecoverInfo(db.getId(), -1L, -1L); + GlobalStateMgr.getCurrentState().getEditLog().logRecoverDb(recoverInfo); } - 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); - } + public void replayRecoverDatabase(RecoverInfo info) { + GlobalStateMgr globalStateMgr = GlobalStateMgr.getCurrentState(); - String tableName = recoverStmt.getTableName(); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); + long dbId = info.getDbId(); + Database db = recycleBin.replayRecoverDatabase(dbId); + globalStateMgr.tryLock(true); try { - Table table = getTable(db.getFullName(), tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (!table.isOlapOrCloudNativeTable()) { - throw new DdlException("table[" + tableName + "] is not OLAP table or LAKE table"); - } - OlapTable olapTable = (OlapTable) table; - - String partitionName = recoverStmt.getPartitionName(); - if (olapTable.getPartition(partitionName) != null) { - throw new DdlException("partition[" + partitionName + "] already exist in table[" + tableName + "]"); - } - - recycleBin.recoverPartition(db.getId(), olapTable, partitionName); + unprotectCreateDb(db); + LOG.info("finish replay create db, name: {}, id: {}", db.getOriginName(), db.getId()); } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); + globalStateMgr.unlock(); } - } - - public void replayEraseDatabase(long dbId) { - recycleBin.replayEraseDatabase(dbId); - } - - public void replayRecoverDatabase(RecoverInfo info) { - long dbId = info.getDbId(); - Database db = recycleBin.replayRecoverDatabase(dbId); - - // add db to globalStateMgr - replayCreateDb(db); LOG.info("replay recover db[{}], name: {}", dbId, db.getOriginName()); } - 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); - } - - DatabaseInfo dbInfo = new DatabaseInfo(db.getFullName(), "", stmt.getQuota(), stmt.getQuotaType()); - GlobalStateMgr.getCurrentState().getEditLog().logAlterDb(dbInfo); + @Override + public void alterDatabaseQuota(DatabaseInfo dbInfo) { + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbInfo.getDbName()); Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.WRITE); @@ -687,6 +279,8 @@ public void alterDatabaseQuota(AlterDatabaseQuotaStmt stmt) throws DdlException } finally { locker.unLockDatabase(db.getId(), LockType.WRITE); } + + GlobalStateMgr.getCurrentState().getEditLog().logAlterDb(dbInfo); } public void replayAlterDatabaseQuota(DatabaseInfo dbInfo) { @@ -704,54 +298,15 @@ public void replayAlterDatabaseQuota(DatabaseInfo dbInfo) { } } - public void renameDatabase(AlterDatabaseRenameStatement stmt) throws DdlException { - String fullDbName = stmt.getDbName(); - String newFullDbName = stmt.getNewDbName(); - - if (fullDbName.equals(newFullDbName)) { - throw new DdlException("Same database name"); - } - - Database db; - if (!tryLock(false)) { - throw new DdlException("Failed to acquire globalStateMgr lock. Try again"); - } - try { - // check if db exists - db = fullNameToDb.get(fullDbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, fullDbName); - } - - // check if name is already used - if (fullNameToDb.get(newFullDbName) != null) { - throw new DdlException("Database name[" + newFullDbName + "] is already used"); - } - // 1. rename db - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - db.setName(newFullDbName); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - - // 2. add to meta. check again - fullNameToDb.remove(fullDbName); - fullNameToDb.put(newFullDbName, db); - - DatabaseInfo dbInfo = - new DatabaseInfo(fullDbName, newFullDbName, -1L, AlterDatabaseQuotaStmt.QuotaType.NONE); - GlobalStateMgr.getCurrentState().getEditLog().logDatabaseRename(dbInfo); - } finally { - unlock(); - } - - LOG.info("rename database[{}] to [{}], id: {}", fullDbName, newFullDbName, db.getId()); + @Override + public void renameDatabase(String dbName, String newDbName) { + replayRenameDatabase(dbName, newDbName); + DatabaseInfo dbInfo = new DatabaseInfo(dbName, newDbName, -1L, AlterDatabaseQuotaStmt.QuotaType.NONE); + GlobalStateMgr.getCurrentState().getEditLog().logDatabaseRename(dbInfo); } public void replayRenameDatabase(String dbName, String newDbName) { - tryLock(true); + GlobalStateMgr.getCurrentState().tryLock(true); try { Database db = fullNameToDb.get(dbName); db.setName(newDbName); @@ -760,289 +315,330 @@ public void replayRenameDatabase(String dbName, String newDbName) { LOG.info("replay rename database {} to {}, id: {}", dbName, newDbName, db.getId()); } finally { - unlock(); + GlobalStateMgr.getCurrentState().unlock(); } } - /** - * Following is the step to create an olap table: - * 1. create columns - * 2. create partition info - * 3. create distribution info - * 4. set table id and base index id - * 5. set bloom filter columns - * 6. set and build TableProperty includes: - * 6.1. dynamicProperty - * 6.2. replicationNum - * 6.3. inMemory - * 7. set index meta - * 8. check colocation properties - * 9. create tablet in BE - * 10. add this table to FE's meta - * 11. add this table to ColocateGroup if necessary - * - * @return whether the table is created - */ @Override - public boolean createTable(CreateTableStmt stmt) throws DdlException { - // check if db exists - Database db = getDb(stmt.getDbName()); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDbName()); - } - - boolean isTemporaryTable = (stmt instanceof CreateTemporaryTableStmt); - // perform the existence check which is cheap before any further heavy operations. - // NOTE: don't even check the quota if already exists. - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - String tableName = stmt.getTableName(); - if (!isTemporaryTable && getTable(db.getFullName(), tableName) != null) { - if (!stmt.isSetIfNotExists()) { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } - LOG.info("create table[{}] which already exists", tableName); - return false; - } - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - - // only internal table should check quota and cluster capacity - if (!stmt.isExternal()) { - // check cluster capacity - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().checkClusterCapacity(); - // check db quota - db.checkQuota(); - } - - AbstractTableFactory tableFactory = TableFactoryProvider.getFactory(stmt.getEngineName()); - if (tableFactory == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_STORAGE_ENGINE, stmt.getEngineName()); - } - - Table table = tableFactory.createTable(this, db, stmt); - String storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr() - .getStorageVolumeIdOfTable(table.getId()); - - try { - onCreate(db, table, storageVolumeId, stmt.isSetIfNotExists()); - } catch (DdlException e) { - if (table.isCloudNativeTable()) { - GlobalStateMgr.getCurrentState().getStorageVolumeMgr().unbindTableToStorageVolume(table.getId()); - } - throw e; - } - return true; + public List listDbNames() { + return Lists.newArrayList(fullNameToDb.keySet()); } @Override - public void createTableLike(CreateTableLikeStmt stmt) throws DdlException { - createTable(stmt.getCreateTableStmt()); + public ConcurrentHashMap getIdToDb() { + return idToDb; } @Override - public void addPartitions(ConnectContext ctx, Database db, String tableName, AddPartitionClause addPartitionClause) - throws DdlException { - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - Table table = getTable(db.getFullName(), tableName); - CatalogUtils.checkTableExist(db, tableName); - CatalogUtils.checkNativeTable(db, table); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - addPartitions(ctx, db, tableName, - addPartitionClause.getResolvedPartitionDescList(), - addPartitionClause.isTempPartition(), - addPartitionClause.getDistributionDesc()); - } - - private OlapTable checkTable(Database db, String tableName) throws DdlException { - CatalogUtils.checkTableExist(db, tableName); - Table table = getTable(db.getFullName(), tableName); - CatalogUtils.checkNativeTable(db, table); - OlapTable olapTable = (OlapTable) table; - CatalogUtils.checkTableState(olapTable, tableName); - return olapTable; + public List getDbIds() { + return Lists.newArrayList(idToDb.keySet()); } - private OlapTable checkTable(Database db, Long tableId) throws DdlException { - Table table = getTable(db.getId(), tableId); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableId); - } - CatalogUtils.checkNativeTable(db, table); - OlapTable olapTable = (OlapTable) table; - CatalogUtils.checkTableState(olapTable, table.getName()); - return olapTable; + @Override + public ConcurrentHashMap getFullNameToDb() { + return fullNameToDb; } - private void checkPartitionType(PartitionInfo partitionInfo) throws DdlException { - PartitionType partitionType = partitionInfo.getType(); - if (!partitionInfo.isRangePartition() && partitionType != PartitionType.LIST) { - throw new DdlException("Only support adding partition to range/list partitioned table"); + @Override + public Database getDb(String name) { + if (name == null) { + return null; } - } - - private DistributionInfo getDistributionInfo(OlapTable olapTable, DistributionDesc distributionDesc) - throws DdlException { - DistributionInfo distributionInfo; - List baseSchema = olapTable.getBaseSchema(); - DistributionInfo defaultDistributionInfo = olapTable.getDefaultDistributionInfo(); - if (distributionDesc != null) { - distributionInfo = distributionDesc.toDistributionInfo(baseSchema); - // for now. we only support modify distribution's bucket num - if (distributionInfo.getType() != defaultDistributionInfo.getType()) { - throw new DdlException("Cannot assign different distribution type. default is: " - + defaultDistributionInfo.getType()); - } - - if (distributionInfo.getType() == DistributionInfo.DistributionInfoType.HASH) { - HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; - List newDistriCols = MetaUtils.getColumnsByColumnIds(olapTable, - hashDistributionInfo.getDistributionColumns()); - List defaultDistriCols = MetaUtils.getColumnsByColumnIds(olapTable, - defaultDistributionInfo.getDistributionColumns()); - if (!newDistriCols.equals(defaultDistriCols)) { - throw new DdlException("Cannot assign hash distribution with different distribution cols. " - + "default is: " + defaultDistriCols); - } - if (hashDistributionInfo.getBucketNum() < 0) { - throw new DdlException("Cannot assign hash distribution buckets less than 0"); - } - } - if (distributionInfo.getType() == DistributionInfo.DistributionInfoType.RANDOM) { - RandomDistributionInfo randomDistributionInfo = (RandomDistributionInfo) distributionInfo; - if (randomDistributionInfo.getBucketNum() < 0) { - throw new DdlException("Cannot assign random distribution buckets less than 0"); - } - } + if (fullNameToDb.containsKey(name)) { + return fullNameToDb.get(name); } else { - distributionInfo = defaultDistributionInfo; - } - return distributionInfo; - } - - private void checkColocation(Database db, OlapTable olapTable, DistributionInfo distributionInfo, - List partitionDescs) - throws DdlException { - if (colocateTableIndex.isColocateTable(olapTable.getId())) { - String fullGroupName = db.getId() + "_" + olapTable.getColocateGroup(); - ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(fullGroupName); - Preconditions.checkNotNull(groupSchema); - groupSchema.checkDistribution(olapTable.getIdToColumn(), distributionInfo); - for (PartitionDesc partitionDesc : partitionDescs) { - groupSchema.checkReplicationNum(partitionDesc.getReplicationNum()); + // This maybe an 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) || + dbName.equalsIgnoreCase(SysDb.DATABASE_NAME)) { + return fullNameToDb.get(dbName.toLowerCase()); } } + return null; } - private void checkDataProperty(List partitionDescs) { - for (PartitionDesc partitionDesc : partitionDescs) { - DataProperty dataProperty = partitionDesc.getPartitionDataProperty(); - Preconditions.checkNotNull(dataProperty); - } + @Override + public Database getDb(long dbId) { + return idToDb.get(dbId); } - private List> createPartitionMap(Database db, OlapTable copiedTable, - List partitionDescs, - HashMap> partitionNameToTabletSet, - Set tabletIdSetForAll, - Set existPartitionNameSet, - long warehouseId) - throws DdlException { - List> partitionList = Lists.newArrayList(); - for (PartitionDesc partitionDesc : partitionDescs) { - long partitionId = getNextId(); - DataProperty dataProperty = partitionDesc.getPartitionDataProperty(); - String partitionName = partitionDesc.getPartitionName(); - if (existPartitionNameSet.contains(partitionName)) { - continue; - } - Long version = partitionDesc.getVersionInfo(); - Set tabletIdSet = Sets.newHashSet(); - - copiedTable.getPartitionInfo().setDataProperty(partitionId, dataProperty); - copiedTable.getPartitionInfo().setTabletType(partitionId, partitionDesc.getTabletType()); - copiedTable.getPartitionInfo().setReplicationNum(partitionId, partitionDesc.getReplicationNum()); - copiedTable.getPartitionInfo().setIsInMemory(partitionId, partitionDesc.isInMemory()); - copiedTable.getPartitionInfo().setDataCacheInfo(partitionId, partitionDesc.getDataCacheInfo()); - - Partition partition = - createPartition(db, copiedTable, partitionId, partitionName, version, tabletIdSet, warehouseId); - - partitionList.add(Pair.create(partition, partitionDesc)); - tabletIdSetForAll.addAll(tabletIdSet); - partitionNameToTabletSet.put(partitionName, tabletIdSet); - } - return partitionList; + @Override + public void createTable(CreateTableInfo createTableInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logCreateTable(createTableInfo); } - private void checkIfMetaChange(OlapTable olapTable, OlapTable copiedTable, String tableName) throws DdlException { - // rollup index may be added or dropped during add partition operation. - // schema may be changed during add partition operation. - boolean metaChanged = false; - if (olapTable.getIndexNameToId().size() != copiedTable.getIndexNameToId().size()) { - metaChanged = true; - } else { - // compare schemaHash - for (Map.Entry entry : olapTable.getIndexIdToMeta().entrySet()) { - long indexId = entry.getKey(); - if (!copiedTable.getIndexIdToMeta().containsKey(indexId)) { - metaChanged = true; - break; - } - if (copiedTable.getIndexIdToMeta().get(indexId).getSchemaHash() != - entry.getValue().getSchemaHash()) { - metaChanged = true; - break; - } - } - } - - if (olapTable.getDefaultDistributionInfo().getType() != - copiedTable.getDefaultDistributionInfo().getType()) { - metaChanged = true; + public void replayCreateTable(CreateTableInfo info) { + Table table = info.getTable(); + Database db = this.fullNameToDb.get(info.getDbName()); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + db.registerTableUnlocked(table); + if (table.isTemporaryTable()) { + TemporaryTableMgr temporaryTableMgr = GlobalStateMgr.getCurrentState().getTemporaryTableMgr(); + UUID sessionId = ((OlapTable) table).getSessionId(); + temporaryTableMgr.addTemporaryTable(sessionId, db.getId(), table.getName(), table.getId()); + } + table.onReload(); + } catch (Throwable e) { + LOG.error("replay create table failed: {}", table, e); + // Rethrow, we should not eat the exception when replaying editlog. + throw e; + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + + if (!isCheckpointThread()) { + // add to inverted index + if (table.isOlapOrCloudNativeTable() || table.isMaterializedView()) { + TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + OlapTable olapTable = (OlapTable) table; + long dbId = db.getId(); + long tableId = table.getId(); + for (PhysicalPartition partition : olapTable.getAllPhysicalPartitions()) { + long physicalPartitionId = partition.getId(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( + partition.getParentId()).getStorageMedium(); + for (MaterializedIndex mIndex : partition + .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + long indexId = mIndex.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partition.getParentId(), physicalPartitionId, + indexId, schemaHash, medium, table.isCloudNativeTableOrMaterializedView()); + for (Tablet tablet : mIndex.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.addTablet(tabletId, tabletMeta); + if (tablet instanceof LocalTablet) { + for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { + invertedIndex.addReplica(tabletId, replica); + } + } + } + } + } // end for partitions + + DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), olapTable); + } } - if (metaChanged) { - throw new DdlException("Table[" + tableName + "]'s meta has been changed. try again."); + // If user upgrades from 3.0, the storage volume id will be null + if (table.isCloudNativeTableOrMaterializedView() && info.getStorageVolumeId() != null) { + GlobalStateMgr.getCurrentState().getStorageVolumeMgr() + .replayBindTableToStorageVolume(info.getStorageVolumeId(), table.getId()); } } - private void updatePartitionInfo(PartitionInfo partitionInfo, List> partitionList, - Set existPartitionNameSet, boolean isTempPartition, - OlapTable olapTable) - throws DdlException { - if (partitionInfo instanceof RangePartitionInfo) { - RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; - rangePartitionInfo.handleNewRangePartitionDescs(olapTable.getIdToColumn(), - partitionList, existPartitionNameSet, isTempPartition); - } else if (partitionInfo instanceof ListPartitionInfo) { - ListPartitionInfo listPartitionInfo = (ListPartitionInfo) partitionInfo; - listPartitionInfo.handleNewListPartitionDescs(olapTable.getIdToColumn(), - partitionList, existPartitionNameSet, isTempPartition); - } else { - throw new DdlException("Only support adding partition to range/list partitioned table"); + @Override + public void dropTable(DropInfo dropInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logDropTable(dropInfo); + } + + public void replayDropTable(Database db, long tableId, boolean isForceDrop) { + Table table; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + table = getTable(db.getId(), tableId); + if (table.isTemporaryTable()) { + table = db.unprotectDropTemporaryTable(tableId, isForceDrop, false); + UUID sessionId = ((OlapTable) table).getSessionId(); + TemporaryTableMgr temporaryTableMgr = GlobalStateMgr.getCurrentState().getTemporaryTableMgr(); + temporaryTableMgr.dropTemporaryTable(sessionId, db.getId(), table.getName()); + } else { + table = db.unprotectDropTable(tableId, isForceDrop, true); + } + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + if (table != null && isForceDrop) { + table.delete(db.getId(), true); } + } - if (isTempPartition) { - for (Pair entry : partitionList) { - Partition partition = entry.first; - if (!existPartitionNameSet.contains(partition.getName())) { - olapTable.addTempPartition(partition); + @Override + public void renameTable(TableInfo tableInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logTableRename(tableInfo); + } + + public void replayRenameTable(TableInfo tableInfo) { + long dbId = tableInfo.getDbId(); + long tableId = tableInfo.getTableId(); + String newTableName = tableInfo.getNewTableName(); + + Database db = getDb(dbId); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable table = (OlapTable) getTable(db.getId(), tableId); + String tableName = table.getName(); + db.dropTable(tableName); + table.setName(newTableName); + db.registerTableUnlocked(table); + inactiveRelatedMaterializedView(db, table, + MaterializedViewExceptions.inactiveReasonForBaseTableRenamed(tableName)); + + LOG.info("replay rename table[{}] to {}, tableId: {}", tableName, newTableName, table.getId()); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + @Override + public void truncateTable(TruncateTableInfo info) { + GlobalStateMgr.getCurrentState().getEditLog().logTruncateTable(info); + } + + public void truncateTableInternal(OlapTable olapTable, List newPartitions, + boolean isEntireTable, boolean isReplay) { + // use new partitions to replace the old ones. + Set oldTablets = Sets.newHashSet(); + for (Partition newPartition : newPartitions) { + Partition oldPartition = olapTable.replacePartition(newPartition); + for (PhysicalPartition physicalPartition : oldPartition.getSubPartitions()) { + // save old tablets to be removed + for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + // let HashSet do the deduplicate work + oldTablets.addAll(index.getTablets()); } } - } else { - for (Pair entry : partitionList) { - Partition partition = entry.first; - if (!existPartitionNameSet.contains(partition.getName())) { - olapTable.addPartition(partition); + } + + if (isEntireTable) { + // drop all temp partitions + olapTable.dropAllTempPartitions(); + } + + // remove the tablets in old partitions + for (Tablet tablet : oldTablets) { + TabletInvertedIndex index = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + index.deleteTablet(tablet.getId()); + // Ensure that only the leader records truncate information. + // TODO(yangzaorang): the information will be lost when failover occurs. The probability of this case + // happening is small, and the trash data will be deleted by BE anyway, but we need to find a better + // solution. + if (!isReplay) { + index.markTabletForceDelete(tablet); + } + } + } + + public void replayTruncateTable(TruncateTableInfo info) { + Database db = getDb(info.getDbId()); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) getTable(db.getId(), info.getTblId()); + truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable(), true); + + if (!GlobalStateMgr.isCheckpointThread()) { + // add tablet to inverted index + TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + for (Partition partition : info.getPartitions()) { + long partitionId = partition.getId(); + TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( + partitionId).getStorageMedium(); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + for (MaterializedIndex mIndex : physicalPartition.getMaterializedIndices( + MaterializedIndex.IndexExtState.ALL)) { + long indexId = mIndex.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(), + physicalPartition.getId(), indexId, schemaHash, medium, + olapTable.isCloudNativeTableOrMaterializedView()); + for (Tablet tablet : mIndex.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.addTablet(tabletId, tabletMeta); + if (olapTable.isOlapTable()) { + for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { + invertedIndex.addReplica(tabletId, replica); + } + } + } + } + } } } + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); + } + } + + @Override + public void modifyTableProperty(Database db, OlapTable table, Map properties, short operationType) { + ModifyTablePropertyOperationLog info = + new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); + + switch (operationType) { + case OperationType.OP_MODIFY_IN_MEMORY: + GlobalStateMgr.getCurrentState().getEditLog().logModifyInMemory(info); + break; + case OperationType.OP_MODIFY_WRITE_QUORUM: + GlobalStateMgr.getCurrentState().getEditLog().logModifyWriteQuorum(info); + break; + } + } + + @Override + public void alterTable(ModifyTablePropertyOperationLog log) { + GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(log); + } + + @Override + public List listTableNames(String dbName) { + Database database = getDb(dbName); + if (database != null) { + return database.getTables().stream() + .map(Table::getName).collect(Collectors.toList()); + } else { + throw new StarRocksConnectorException("Database " + dbName + " doesn't exist"); + } + } + + @Override + public List
getTables(Long dbId) { + Database database = getDb(dbId); + if (database == null) { + return Collections.emptyList(); + } else { + return database.getTables(); + } + } + + @Override + public Table getTable(String dbName, String tblName) { + Database database = getDb(dbName); + if (database == null) { + return null; + } + return database.getTable(tblName); + } + + @Override + public Table getTable(Long dbId, Long tableId) { + Database database = getDb(dbId); + if (database == null) { + return null; + } + return database.getTable(tableId); + } + + @Override + public void addPartitionLog(Database db, OlapTable olapTable, List partitionDescs, + boolean isTempPartition, PartitionInfo partitionInfo, + List partitionList, Set existPartitionNameSet) + throws DdlException { + PartitionType partitionType = partitionInfo.getType(); + if (partitionInfo.isRangePartition()) { + addRangePartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, + existPartitionNameSet); + } else if (partitionType == PartitionType.LIST) { + addListPartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, + existPartitionNameSet); + } else { + throw new DdlException("Only support adding partition log to range/list partitioned table"); } } @@ -1128,175 +724,6 @@ public void addListPartitionLog(Database db, OlapTable olapTable, List partitionDescs, - boolean isTempPartition, PartitionInfo partitionInfo, - List partitionList, Set existPartitionNameSet) - throws DdlException { - PartitionType partitionType = partitionInfo.getType(); - if (partitionInfo.isRangePartition()) { - addRangePartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, - existPartitionNameSet); - } else if (partitionType == PartitionType.LIST) { - addListPartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, - existPartitionNameSet); - } else { - throw new DdlException("Only support adding partition log to range/list partitioned table"); - } - } - - private void addSubPartitionLog(Database db, OlapTable olapTable, Partition partition, - List subPartitioins) throws DdlException { - List partitionInfoV2List = Lists.newArrayList(); - for (PhysicalPartition subPartition : subPartitioins) { - if (subPartition instanceof PhysicalPartitionImpl) { - PhysicalPartitionPersistInfoV2 info = new PhysicalPartitionPersistInfoV2(db.getId(), olapTable.getId(), - partition.getId(), (PhysicalPartitionImpl) subPartition); - partitionInfoV2List.add(info); - } - } - - AddSubPartitionsInfoV2 infos = new AddSubPartitionsInfoV2(partitionInfoV2List); - GlobalStateMgr.getCurrentState().getEditLog().logAddSubPartitions(infos); - - for (PhysicalPartition subPartition : subPartitioins) { - LOG.info("succeed in creating sub partitions[{}]", subPartition); - } - - } - - private void cleanExistPartitionNameSet(Set existPartitionNameSet, - HashMap> partitionNameToTabletSet) { - for (String partitionName : existPartitionNameSet) { - Set existPartitionTabletSet = partitionNameToTabletSet.get(partitionName); - if (existPartitionTabletSet == null) { - // should not happen - continue; - } - for (Long tabletId : existPartitionTabletSet) { - // createPartitionWithIndices create duplicate tablet that if not exists scenario - // so here need to clean up those created tablets which partition already exists from invert index - GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); - } - } - } - - private void cleanTabletIdSetForAll(Set tabletIdSetForAll) { - // Cleanup of shards for LakeTable is taken care by ShardDeleter - for (Long tabletId : tabletIdSetForAll) { - GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); - } - } - - private void checkPartitionNum(OlapTable olapTable) throws DdlException { - if (olapTable.getNumberOfPartitions() > Config.max_partition_number_per_table) { - throw new DdlException("Table " + olapTable.getName() + " created partitions exceeded the maximum limit: " + - Config.max_partition_number_per_table + ". You can modify this restriction on by setting" + - " max_partition_number_per_table larger."); - } - } - - private void addPartitions(ConnectContext ctx, Database db, String tableName, List partitionDescs, - boolean isTempPartition, DistributionDesc distributionDesc) throws DdlException { - DistributionInfo distributionInfo; - OlapTable olapTable; - OlapTable copiedTable; - - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - Set checkExistPartitionName = Sets.newConcurrentHashSet(); - try { - olapTable = checkTable(db, tableName); - - // get partition info - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - - // check partition type - checkPartitionType(partitionInfo); - - // check partition num - checkPartitionNum(olapTable); - - // get distributionInfo - distributionInfo = getDistributionInfo(olapTable, distributionDesc).copy(); - olapTable.inferDistribution(distributionInfo); - - // check colocation - checkColocation(db, olapTable, distributionInfo, partitionDescs); - copiedTable = getShadowCopyTable(olapTable); - copiedTable.setDefaultDistributionInfo(distributionInfo); - checkExistPartitionName = CatalogUtils.checkPartitionNameExistForAddPartitions(olapTable, partitionDescs); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - - Preconditions.checkNotNull(distributionInfo); - Preconditions.checkNotNull(olapTable); - Preconditions.checkNotNull(copiedTable); - - // create partition outside db lock - checkDataProperty(partitionDescs); - - Set tabletIdSetForAll = Sets.newHashSet(); - HashMap> partitionNameToTabletSet = Maps.newHashMap(); - try { - // create partition list - List> newPartitions = - createPartitionMap(db, copiedTable, partitionDescs, partitionNameToTabletSet, tabletIdSetForAll, - checkExistPartitionName, ctx.getCurrentWarehouseId()); - - // build partitions - List partitionList = newPartitions.stream().map(x -> x.first).collect(Collectors.toList()); - buildPartitions(db, copiedTable, partitionList.stream().map(Partition::getSubPartitions) - .flatMap(p -> p.stream()).collect(Collectors.toList()), ctx.getCurrentWarehouseId()); - - // check again - if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { - throw new DdlException("db " + db.getFullName() - + "(" + db.getId() + ") has been dropped"); - } - Set existPartitionNameSet = Sets.newHashSet(); - try { - olapTable = checkTable(db, tableName); - existPartitionNameSet = CatalogUtils.checkPartitionNameExistForAddPartitions(olapTable, - partitionDescs); - if (existPartitionNameSet.size() > 0) { - for (String partitionName : existPartitionNameSet) { - LOG.info("add partition[{}] which already exists", partitionName); - } - } - - // check if meta changed - checkIfMetaChange(olapTable, copiedTable, tableName); - - // get partition info - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - - // check partition type - checkPartitionType(partitionInfo); - - // update partition info - updatePartitionInfo(partitionInfo, newPartitions, existPartitionNameSet, isTempPartition, olapTable); - - try { - colocateTableIndex.updateLakeTableColocationInfo(olapTable, true /* isJoin */, - null /* expectGroupId */); - } catch (DdlException e) { - LOG.info("table {} update colocation info failed when add partition, {}", olapTable.getId(), e.getMessage()); - } - - // add partition log - addPartitionLog(db, olapTable, partitionDescs, isTempPartition, partitionInfo, partitionList, - existPartitionNameSet); - } finally { - cleanExistPartitionNameSet(existPartitionNameSet, partitionNameToTabletSet); - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } catch (DdlException e) { - cleanTabletIdSetForAll(tabletIdSetForAll); - throw e; - } - } - public void replayAddPartition(PartitionPersistInfoV2 info) throws DdlException { Database db = this.getDb(info.getDbId()); Locker locker = new Locker(); @@ -1334,18 +761,21 @@ public void replayAddPartition(PartitionPersistInfoV2 info) throws DdlException if (!isCheckpointThread()) { // add to inverted index TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - long indexId = index.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), - index.getId(), schemaHash, info.getDataProperty().getStorageMedium()); - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - // modify some logic - if (tablet instanceof LocalTablet) { - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + for (MaterializedIndex index : + physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + long indexId = index.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), + index.getId(), schemaHash, info.getDataProperty().getStorageMedium()); + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.addTablet(tabletId, tabletMeta); + // modify some logic + if (tablet instanceof LocalTablet) { + for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { + invertedIndex.addReplica(tabletId, replica); + } } } } @@ -1356,151 +786,9 @@ public void replayAddPartition(PartitionPersistInfoV2 info) throws DdlException } } - public void replayAddPartition(PartitionPersistInfo info) throws DdlException { - Database db = this.getDb(info.getDbId()); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), info.getTableId()); - Partition partition = info.getPartition(); - - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - if (info.isTempPartition()) { - olapTable.addTempPartition(partition); - } else { - olapTable.addPartition(partition); - } - - if (partitionInfo.isRangePartition()) { - ((RangePartitionInfo) partitionInfo).unprotectHandleNewSinglePartitionDesc(partition.getId(), - info.isTempPartition(), info.getRange(), info.getDataProperty(), info.getReplicationNum(), - info.isInMemory()); - } else { - partitionInfo.addPartition( - partition.getId(), info.getDataProperty(), info.getReplicationNum(), info.isInMemory()); - } - if (!isCheckpointThread()) { - // add to inverted index - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - long indexId = index.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), - index.getId(), schemaHash, info.getDataProperty().getStorageMedium()); - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - @Override - public void dropPartition(Database db, Table table, DropPartitionClause clause) throws DdlException { - CatalogUtils.checkTableExist(db, table.getName()); - Locker locker = new Locker(); - OlapTable olapTable = (OlapTable) table; - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); - } - if (!partitionInfo.isRangePartition() && partitionInfo.getType() != PartitionType.LIST) { - throw new DdlException("Alter table [" + olapTable.getName() + "] failed. Not a partitioned table"); - } - boolean isTempPartition = clause.isTempPartition(); - - List existPartitions = Lists.newArrayList(); - List notExistPartitions = Lists.newArrayList(); - for (String partitionName : clause.getResolvedPartitionNames()) { - if (olapTable.checkPartitionNameExist(partitionName, isTempPartition)) { - existPartitions.add(partitionName); - } else { - notExistPartitions.add(partitionName); - } - } - if (CollectionUtils.isNotEmpty(notExistPartitions)) { - if (clause.isSetIfExists()) { - LOG.info("drop partition[{}] which does not exist", notExistPartitions); - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_DROP_PARTITION_NON_EXISTENT, notExistPartitions); - } - } - if (CollectionUtils.isEmpty(existPartitions)) { - return; - } - for (String partitionName : existPartitions) { - // drop - if (isTempPartition) { - olapTable.dropTempPartition(partitionName, true); - } else { - Partition partition = olapTable.getPartition(partitionName); - if (!clause.isForceDrop()) { - if (partition != null) { - if (stateMgr.getGlobalTransactionMgr() - .existCommittedTxns(db.getId(), olapTable.getId(), partition.getId())) { - throw new DdlException( - "There are still some transactions in the COMMITTED state waiting to be completed." + - " The partition [" + partitionName + - "] cannot be dropped. If you want to forcibly drop(cannot be recovered)," + - " please use \"DROP PARTITION FORCE\"."); - } - } - } - Range partitionRange = null; - if (partition != null) { - GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); - if (partitionInfo instanceof RangePartitionInfo) { - partitionRange = ((RangePartitionInfo) partitionInfo).getRange(partition.getId()); - } - } - - olapTable.dropPartition(db.getId(), partitionName, clause.isForceDrop()); - if (olapTable instanceof MaterializedView) { - MaterializedView mv = (MaterializedView) olapTable; - SyncPartitionUtils.dropBaseVersionMeta(mv, partitionName, partitionRange); - } - } - } - if (!isTempPartition) { - try { - for (MvId mvId : olapTable.getRelatedMaterializedViews()) { - MaterializedView materializedView = (MaterializedView) getTable(db.getId(), mvId.getId()); - if (materializedView != null && materializedView.isLoadTriggeredRefresh()) { - GlobalStateMgr.getCurrentState().getLocalMetastore().refreshMaterializedView( - db.getFullName(), materializedView.getName(), false, null, - Constants.TaskRunPriority.NORMAL.value(), true, false); - } - } - } catch (MetaNotFoundException e) { - throw new DdlException("fail to refresh materialized views when dropping partition", e); - } - } - long dbId = db.getId(); - long tableId = olapTable.getId(); - EditLog editLog = GlobalStateMgr.getCurrentState().getEditLog(); - - if (clause.getPartitionName() != null) { - String partitionName = clause.getPartitionName(); - DropPartitionInfo info = new DropPartitionInfo(dbId, tableId, partitionName, isTempPartition, clause.isForceDrop()); - editLog.logDropPartition(info); - LOG.info("succeed in dropping partition[{}], is temp : {}, is force : {}", partitionName, isTempPartition, - clause.isForceDrop()); - } else { - DropPartitionsInfo info = - new DropPartitionsInfo(dbId, tableId, isTempPartition, clause.isForceDrop(), existPartitions); - editLog.logDropPartitions(info); - LOG.info("succeed in dropping partitions[{}], is temp : {}, is force : {}", existPartitions, isTempPartition, - clause.isForceDrop()); - } - + public void dropPartition(DropPartitionInfo dropPartitionInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logDropPartition(dropPartitionInfo); } public void replayDropPartition(DropPartitionInfo info) { @@ -1519,6 +807,11 @@ public void replayDropPartition(DropPartitionInfo info) { } } + @Override + public void dropPartitions(DropPartitionsInfo dropPartitionInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logDropPartitions(dropPartitionInfo); + } + public void replayDropPartitions(DropPartitionsInfo info) { Database db = this.getDb(info.getDbId()); Locker locker = new Locker(); @@ -1538,2992 +831,39 @@ public void replayDropPartitions(DropPartitionsInfo info) { } else { olapTable.dropPartition(dbId, partitionName, isForceDrop); } - }); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayErasePartition(long partitionId) throws DdlException { - recycleBin.replayErasePartition(partitionId); - } - - public void replayRecoverPartition(RecoverInfo info) { - long dbId = info.getDbId(); - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - Table table = getTable(db.getId(), info.getTableId()); - recycleBin.replayRecoverPartition((OlapTable) table, info.getPartitionId()); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - private PhysicalPartition createPhysicalPartition(String name, Database db, OlapTable olapTable, - Partition partition, long warehouseId) throws DdlException { - long partitionId = partition.getId(); - DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo().copy(); - olapTable.inferDistribution(distributionInfo); - // create sub partition - Map indexMap = new HashMap<>(); - for (long indexId : olapTable.getIndexIdToMeta().keySet()) { - MaterializedIndex rollup = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); - indexMap.put(indexId, rollup); - } - - Long id = GlobalStateMgr.getCurrentState().getNextId(); - // physical partitions in the same logical partition use the same shard_group_id, - // so that the shards of this logical partition are more evenly distributed. - long shardGroupId = partition.getShardGroupId(); - - if (name == null) { - name = partition.generatePhysicalPartitionName(id); - } - PhysicalPartitionImpl physicalParition = new PhysicalPartitionImpl( - id, name, partition.getId(), shardGroupId, indexMap.get(olapTable.getBaseIndexId())); - - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - short replicationNum = partitionInfo.getReplicationNum(partitionId); - TStorageMedium storageMedium = partitionInfo.getDataProperty(partitionId).getStorageMedium(); - for (Map.Entry entry : indexMap.entrySet()) { - long indexId = entry.getKey(); - MaterializedIndex index = entry.getValue(); - MaterializedIndexMeta indexMeta = olapTable.getIndexIdToMeta().get(indexId); - Set tabletIdSet = new HashSet<>(); - - // create tablets - TabletMeta tabletMeta = - new TabletMeta(db.getId(), olapTable.getId(), id, indexId, indexMeta.getSchemaHash(), - storageMedium, olapTable.isCloudNativeTableOrMaterializedView()); - - if (olapTable.isCloudNativeTableOrMaterializedView()) { - createLakeTablets(olapTable, id, shardGroupId, index, distributionInfo, - tabletMeta, tabletIdSet, warehouseId); - } else { - createOlapTablets(olapTable, index, Replica.ReplicaState.NORMAL, distributionInfo, - physicalParition.getVisibleVersion(), replicationNum, tabletMeta, tabletIdSet); - } - if (index.getId() != olapTable.getBaseIndexId()) { - // add rollup index to partition - physicalParition.createRollupIndex(index); - } - } - - return physicalParition; - } - - public void addSubPartitions(Database db, OlapTable table, Partition partition, - int numSubPartition, long warehouseId) throws DdlException { - try { - table.setAutomaticBucketing(true); - addSubPartitions(db, table, partition, numSubPartition, null, warehouseId); - } finally { - table.setAutomaticBucketing(false); - } - } - - private void addSubPartitions(Database db, OlapTable table, Partition partition, - int numSubPartition, String[] subPartitionNames, long warehouseId) throws DdlException { - OlapTable olapTable; - OlapTable copiedTable; - - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - olapTable = checkTable(db, table.getId()); - - if (partition.getDistributionInfo().getType() != DistributionInfo.DistributionInfoType.RANDOM) { - throw new DdlException("Only support adding physical partition to random distributed table"); - } - - copiedTable = getShadowCopyTable(olapTable); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - - Preconditions.checkNotNull(olapTable); - Preconditions.checkNotNull(copiedTable); - - List subPartitions = new ArrayList<>(); - // create physical partition - for (int i = 0; i < numSubPartition; i++) { - String name = subPartitionNames != null && subPartitionNames.length > i ? subPartitionNames[i] : null; - PhysicalPartition subPartition = createPhysicalPartition(name, db, copiedTable, partition, warehouseId); - subPartitions.add(subPartition); - } - - // build partitions - buildPartitions(db, copiedTable, subPartitions, warehouseId); - - // check again - if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { - throw new DdlException("db " + db.getFullName() - + "(" + db.getId() + ") has been dropped"); - } - try { - // check if meta changed - checkIfMetaChange(olapTable, copiedTable, table.getName()); - - for (PhysicalPartition subPartition : subPartitions) { - // add sub partition - partition.addSubPartition(subPartition); - olapTable.addPhysicalPartition(subPartition); - } - - olapTable.setShardGroupChanged(true); - - // add partition log - addSubPartitionLog(db, olapTable, partition, subPartitions); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayAddSubPartition(PhysicalPartitionPersistInfoV2 info) throws DdlException { - Database db = this.getDb(info.getDbId()); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), info.getTableId()); - Partition partition = olapTable.getPartition(info.getPartitionId()); - PhysicalPartition physicalPartition = info.getPhysicalPartition(); - partition.addSubPartition(physicalPartition); - olapTable.addPhysicalPartition(physicalPartition); - - if (!isCheckpointThread()) { - // add to inverted index - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - long indexId = index.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), info.getPartitionId(), - physicalPartition.getId(), index.getId(), schemaHash, olapTable.getPartitionInfo().getDataProperty( - info.getPartitionId()).getStorageMedium(), false); - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - // modify some logic - if (tablet instanceof LocalTablet) { - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } - } - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - Partition createPartition(Database db, OlapTable table, long partitionId, String partitionName, - Long version, Set tabletIdSet, long warehouseId) throws DdlException { - DistributionInfo distributionInfo = table.getDefaultDistributionInfo().copy(); - table.inferDistribution(distributionInfo); - - return createPartition(db, table, partitionId, partitionName, version, tabletIdSet, distributionInfo, warehouseId); - } - - Partition createPartition(Database db, OlapTable table, long partitionId, String partitionName, - Long version, Set tabletIdSet, DistributionInfo distributionInfo, - long warehouseId) throws DdlException { - PartitionInfo partitionInfo = table.getPartitionInfo(); - Map indexMap = new HashMap<>(); - for (long indexId : table.getIndexIdToMeta().keySet()) { - MaterializedIndex rollup = new MaterializedIndex(indexId, MaterializedIndex.IndexState.NORMAL); - indexMap.put(indexId, rollup); - } - - // create shard group - long shardGroupId = 0; - if (table.isCloudNativeTableOrMaterializedView()) { - shardGroupId = GlobalStateMgr.getCurrentState().getStarOSAgent(). - createShardGroup(db.getId(), table.getId(), partitionId); - } - - Partition partition = - new Partition(partitionId, partitionName, indexMap.get(table.getBaseIndexId()), - distributionInfo, shardGroupId); - // version - if (version != null) { - partition.updateVisibleVersion(version); - } - - short replicationNum = partitionInfo.getReplicationNum(partitionId); - TStorageMedium storageMedium = partitionInfo.getDataProperty(partitionId).getStorageMedium(); - for (Map.Entry entry : indexMap.entrySet()) { - long indexId = entry.getKey(); - MaterializedIndex index = entry.getValue(); - MaterializedIndexMeta indexMeta = table.getIndexIdToMeta().get(indexId); - - // create tablets - TabletMeta tabletMeta = - new TabletMeta(db.getId(), table.getId(), partitionId, indexId, indexMeta.getSchemaHash(), - storageMedium, table.isCloudNativeTableOrMaterializedView()); - - if (table.isCloudNativeTableOrMaterializedView()) { - createLakeTablets(table, partitionId, shardGroupId, index, distributionInfo, - tabletMeta, tabletIdSet, warehouseId); - } else { - createOlapTablets(table, index, Replica.ReplicaState.NORMAL, distributionInfo, - partition.getVisibleVersion(), replicationNum, tabletMeta, tabletIdSet); - } - if (index.getId() != table.getBaseIndexId()) { - // add rollup index to partition - partition.createRollupIndex(index); - } - } - return partition; - } - - void buildPartitions(Database db, OlapTable table, List partitions, long warehouseId) - throws DdlException { - if (partitions.isEmpty()) { - return; - } - int numAliveNodes = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getAliveBackendNumber(); - - if (RunMode.isSharedDataMode()) { - numAliveNodes = 0; - List computeNodeIds = GlobalStateMgr.getCurrentState().getWarehouseMgr().getAllComputeNodeIds(warehouseId); - for (long nodeId : computeNodeIds) { - if (GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getBackendOrComputeNode(nodeId).isAlive()) { - ++numAliveNodes; - } - } - } - if (numAliveNodes == 0) { - if (RunMode.isSharedDataMode()) { - throw new DdlException("no alive compute nodes"); - } else { - throw new DdlException("no alive backends"); - } - } - - int numReplicas = 0; - for (PhysicalPartition partition : partitions) { - numReplicas += partition.storageReplicaCount(); - } - - try { - GlobalStateMgr.getCurrentState().getConsistencyChecker().addCreatingTableId(table.getId()); - if (numReplicas > Config.create_table_max_serial_replicas) { - LOG.info("start to build {} partitions concurrently for table {}.{} with {} replicas", - partitions.size(), db.getFullName(), table.getName(), numReplicas); - TabletTaskExecutor.buildPartitionsConcurrently( - db.getId(), table, partitions, numReplicas, numAliveNodes, warehouseId); - } else { - LOG.info("start to build {} partitions sequentially for table {}.{} with {} replicas", - partitions.size(), db.getFullName(), table.getName(), numReplicas); - TabletTaskExecutor.buildPartitionsSequentially( - db.getId(), table, partitions, numReplicas, numAliveNodes, warehouseId); - } - } finally { - GlobalStateMgr.getCurrentState().getConsistencyChecker().deleteCreatingTableId(table.getId()); - } - } - - /* - * generate and check columns' order and key's existence - */ - void validateColumns(List columns) throws DdlException { - if (columns.isEmpty()) { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_MUST_HAVE_COLUMNS); - } - - boolean encounterValue = false; - boolean hasKey = false; - for (Column column : columns) { - if (column.isKey()) { - if (encounterValue) { - ErrorReport.reportDdlException(ErrorCode.ERR_OLAP_KEY_MUST_BEFORE_VALUE); - } - hasKey = true; - } else { - encounterValue = true; - } - } - - if (!hasKey) { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_MUST_HAVE_KEYS); - } - } - - // only for test - public void setColocateTableIndex(ColocateTableIndex colocateTableIndex) { - this.colocateTableIndex = colocateTableIndex; - } - - public ColocateTableIndex getColocateTableIndex() { - return colocateTableIndex; - } - - public void setLakeStorageInfo(Database db, OlapTable table, String storageVolumeId, Map properties) - throws DdlException { - DataCacheInfo dataCacheInfo = null; - try { - dataCacheInfo = PropertyAnalyzer.analyzeDataCacheInfo(properties); - } catch (AnalysisException e) { - throw new DdlException(e.getMessage()); - } - - // get service shard storage info from StarMgr - FilePathInfo pathInfo = !storageVolumeId.isEmpty() ? - stateMgr.getStarOSAgent().allocateFilePath(storageVolumeId, db.getId(), table.getId()) : - stateMgr.getStarOSAgent().allocateFilePath(db.getId(), table.getId()); - table.setStorageInfo(pathInfo, dataCacheInfo); - } - - public void onCreate(Database db, Table table, String storageVolumeId, boolean isSetIfNotExists) - throws DdlException { - // check database exists again, because database can be dropped when creating table - if (!tryLock(false)) { - throw new DdlException("Failed to acquire globalStateMgr lock. " + - "Try again or increasing value of `catalog_try_lock_timeout_ms` configuration."); - } - - try { - /* - * When creating table or mv, we need to create the tablets and prepare some of the - * metadata first before putting this new table or mv in the database. So after the - * first step, we need to acquire the global lock and double check whether the db still - * exists because it maybe dropped by other concurrent client. And if we don't use the lock - * protection and handle the concurrency properly, the replay of table/mv creation may fail - * on restart or on follower. - * - * After acquire the db lock, we also need to acquire the db lock and write edit log. Since the - * db lock maybe under high contention and IO is busy, current thread can hold the global lock - * for quite a long time and make the other operation waiting for the global lock fail. - * - * So here after the confirmation of existence of modifying database, we release the global lock - * When dropping database, we will set the `exist` field of db object to false. And in the following - * creation process, we will double-check the `exist` field. - */ - if (getDb(db.getId()) == null) { - throw new DdlException("Database has been dropped when creating table/mv/view"); - } - } finally { - unlock(); - } - - if (db.isSystemDatabase()) { - ErrorReport.reportDdlException(ErrorCode.ERR_CANT_CREATE_TABLE, table.getName(), - "cannot create table in system database"); - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - if (!db.isExist()) { - throw new DdlException("Database has been dropped when creating table/mv/view"); - } - - if (!db.registerTableUnlocked(table)) { - if (!isSetIfNotExists) { - table.delete(db.getId(), false); - ErrorReport.reportDdlException(ErrorCode.ERR_CANT_CREATE_TABLE, table.getName(), - "table already exists"); - } else { - LOG.info("Create table[{}] which already exists", table.getName()); - return; - } - } - - // NOTE: The table has been added to the database, and the following procedure cannot throw exception. - LOG.info("Successfully create table: {}-{}, in database: {}-{}", - table.getName(), table.getId(), db.getFullName(), db.getId()); - - CreateTableInfo createTableInfo = new CreateTableInfo(db.getFullName(), table, storageVolumeId); - GlobalStateMgr.getCurrentState().getEditLog().logCreateTable(createTableInfo); - table.onCreate(db); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayCreateTable(CreateTableInfo info) { - Table table = info.getTable(); - Database db = this.fullNameToDb.get(info.getDbName()); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - db.registerTableUnlocked(table); - if (table.isTemporaryTable()) { - TemporaryTableMgr temporaryTableMgr = GlobalStateMgr.getCurrentState().getTemporaryTableMgr(); - UUID sessionId = ((OlapTable) table).getSessionId(); - temporaryTableMgr.addTemporaryTable(sessionId, db.getId(), table.getName(), table.getId()); - } - table.onReload(); - } catch (Throwable e) { - LOG.error("replay create table failed: {}", table, e); - // Rethrow, we should not eat the exception when replaying editlog. - throw e; - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - - if (!isCheckpointThread()) { - // add to inverted index - if (table.isOlapOrCloudNativeTable() || table.isMaterializedView()) { - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - OlapTable olapTable = (OlapTable) table; - long dbId = db.getId(); - long tableId = table.getId(); - for (PhysicalPartition partition : olapTable.getAllPhysicalPartitions()) { - long physicalPartitionId = partition.getId(); - TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( - partition.getParentId()).getStorageMedium(); - for (MaterializedIndex mIndex : partition - .getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - long indexId = mIndex.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partition.getParentId(), physicalPartitionId, - indexId, schemaHash, medium, table.isCloudNativeTableOrMaterializedView()); - for (Tablet tablet : mIndex.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - if (tablet instanceof LocalTablet) { - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } - } // end for partitions - - DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), olapTable); - } - } - - // If user upgrades from 3.0, the storage volume id will be null - if (table.isCloudNativeTableOrMaterializedView() && info.getStorageVolumeId() != null) { - GlobalStateMgr.getCurrentState().getStorageVolumeMgr() - .replayBindTableToStorageVolume(info.getStorageVolumeId(), table.getId()); - } - } - - private void createLakeTablets(OlapTable table, long partitionId, long shardGroupId, MaterializedIndex index, - DistributionInfo distributionInfo, TabletMeta tabletMeta, - Set tabletIdSet, long warehouseId) - throws DdlException { - Preconditions.checkArgument(table.isCloudNativeTableOrMaterializedView()); - - DistributionInfo.DistributionInfoType distributionInfoType = distributionInfo.getType(); - if (distributionInfoType != DistributionInfo.DistributionInfoType.HASH - && distributionInfoType != DistributionInfo.DistributionInfoType.RANDOM) { - throw new DdlException("Unknown distribution type: " + distributionInfoType); - } - - Map properties = new HashMap<>(); - properties.put(LakeTablet.PROPERTY_KEY_TABLE_ID, Long.toString(table.getId())); - properties.put(LakeTablet.PROPERTY_KEY_PARTITION_ID, Long.toString(partitionId)); - properties.put(LakeTablet.PROPERTY_KEY_INDEX_ID, Long.toString(index.getId())); - int bucketNum = distributionInfo.getBucketNum(); - WarehouseManager warehouseManager = GlobalStateMgr.getCurrentState().getWarehouseMgr(); - Optional workerGroupId = warehouseManager.selectWorkerGroupByWarehouseId(warehouseId); - if (workerGroupId.isEmpty()) { - Warehouse warehouse = warehouseManager.getWarehouse(warehouseId); - throw ErrorReportException.report(ErrorCode.ERR_NO_NODES_IN_WAREHOUSE, warehouse.getName()); - } - List shardIds = stateMgr.getStarOSAgent().createShards(bucketNum, - table.getPartitionFilePathInfo(partitionId), table.getPartitionFileCacheInfo(partitionId), shardGroupId, - null, properties, workerGroupId.get()); - for (long shardId : shardIds) { - Tablet tablet = new LakeTablet(shardId); - index.addTablet(tablet, tabletMeta); - tabletIdSet.add(tablet.getId()); - } - } - - private void createOlapTablets(OlapTable table, MaterializedIndex index, Replica.ReplicaState replicaState, - DistributionInfo distributionInfo, long version, short replicationNum, - TabletMeta tabletMeta, Set tabletIdSet) throws DdlException { - Preconditions.checkArgument(replicationNum > 0); - - DistributionInfo.DistributionInfoType distributionInfoType = distributionInfo.getType(); - if (distributionInfoType != DistributionInfo.DistributionInfoType.HASH - && distributionInfoType != DistributionInfo.DistributionInfoType.RANDOM) { - throw new DdlException("Unknown distribution type: " + distributionInfoType); - } - - List> backendsPerBucketSeq = null; - ColocateTableIndex.GroupId groupId = null; - boolean initBucketSeqWithSameOrigNameGroup = false; - boolean isColocateTable = colocateTableIndex.isColocateTable(tabletMeta.getTableId()); - // chooseBackendsArbitrary is true, means this may be the first table of colocation group, - // or this is just a normal table, and we can choose backends arbitrary. - // otherwise, backends should be chosen from backendsPerBucketSeq; - boolean chooseBackendsArbitrary; - - // We should synchronize the creation of colocate tables, otherwise it can have concurrent issues. - // Considering the following situation, - // T1: P1 issues `create colocate table` and finds that there isn't a bucket sequence associated - // with the colocate group, so it will initialize the bucket sequence for the first time - // T2: P2 do the same thing as P1 - // T3: P1 set the bucket sequence for colocate group stored in `ColocateTableIndex` - // T4: P2 also set the bucket sequence, hence overwrite what P1 just wrote - // T5: After P1 creates the colocate table, the actual tablet distribution won't match the bucket sequence - // of the colocate group, and balancer will create a lot of COLOCATE_MISMATCH tasks which shouldn't exist. - if (isColocateTable) { - try { - // Optimization: wait first time, before global lock - colocateTableCreateSyncer.awaitZero(); - // Since we have supported colocate tables in different databases, - // we should use global lock, not db lock. - tryLock(false); - try { - // Wait again, for safety - // We are in global lock, we should have timeout in case holding lock for too long - colocateTableCreateSyncer.awaitZero(Config.catalog_try_lock_timeout_ms, TimeUnit.MILLISECONDS); - // if this is a colocate table, try to get backend seqs from colocation index. - groupId = colocateTableIndex.getGroup(tabletMeta.getTableId()); - backendsPerBucketSeq = colocateTableIndex.getBackendsPerBucketSeq(groupId); - if (backendsPerBucketSeq.isEmpty()) { - List colocateWithGroupsInOtherDb = - colocateTableIndex.getColocateWithGroupsInOtherDb(groupId); - if (!colocateWithGroupsInOtherDb.isEmpty()) { - backendsPerBucketSeq = - colocateTableIndex.getBackendsPerBucketSeq(colocateWithGroupsInOtherDb.get(0)); - initBucketSeqWithSameOrigNameGroup = true; - } - } - chooseBackendsArbitrary = backendsPerBucketSeq == null || backendsPerBucketSeq.isEmpty(); - if (chooseBackendsArbitrary) { - colocateTableCreateSyncer.increment(); - } - } finally { - unlock(); - } - } catch (InterruptedException e) { - LOG.warn("wait for concurrent colocate table creation finish failed, msg: {}", - e.getMessage(), e); - Thread.currentThread().interrupt(); - throw new DdlException("wait for concurrent colocate table creation finish failed", e); - } - } else { - chooseBackendsArbitrary = true; - } - - try { - if (chooseBackendsArbitrary) { - backendsPerBucketSeq = Lists.newArrayList(); - } - for (int i = 0; i < distributionInfo.getBucketNum(); ++i) { - // create a new tablet with random chosen backends - LocalTablet tablet = new LocalTablet(getNextId()); - - // add tablet to inverted index first - index.addTablet(tablet, tabletMeta); - tabletIdSet.add(tablet.getId()); - - // get BackendIds - List chosenBackendIds; - if (chooseBackendsArbitrary) { - // This is the first colocate table in the group, or just a normal table, - // randomly choose backends - if (Config.enable_strict_storage_medium_check) { - chosenBackendIds = - chosenBackendIdBySeq(replicationNum, table.getLocation(), tabletMeta.getStorageMedium()); - } else { - try { - chosenBackendIds = chosenBackendIdBySeq(replicationNum, table.getLocation()); - } catch (DdlException ex) { - throw new DdlException(String.format("%s, table=%s, default_replication_num=%d", - ex.getMessage(), table.getName(), Config.default_replication_num)); - } - } - backendsPerBucketSeq.add(chosenBackendIds); - } else { - // get backends from existing backend sequence - chosenBackendIds = backendsPerBucketSeq.get(i); - } - - // create replicas - for (long backendId : chosenBackendIds) { - long replicaId = getNextId(); - Replica replica = new Replica(replicaId, backendId, replicaState, version, - tabletMeta.getOldSchemaHash()); - tablet.addReplica(replica); - } - Preconditions.checkState(chosenBackendIds.size() == replicationNum, - chosenBackendIds.size() + " vs. " + replicationNum); - } - - // In the following two situations, we should set the bucket seq for colocate group and persist the info, - // 1. This is the first time we add a table to colocate group, and it doesn't have the same original name - // with colocate group in other database. - // 2. It's indeed the first time, but it should colocate with group in other db - // (because of having the same original name), we should use the bucket - // seq of other group to initialize our own. - if ((groupId != null && chooseBackendsArbitrary) || initBucketSeqWithSameOrigNameGroup) { - colocateTableIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeq); - ColocatePersistInfo info = - ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, backendsPerBucketSeq); - GlobalStateMgr.getCurrentState().getEditLog().logColocateBackendsPerBucketSeq(info); - } - } finally { - if (isColocateTable && chooseBackendsArbitrary) { - colocateTableCreateSyncer.decrement(); - } - } - } - - // create replicas for tablet with random chosen backends - private List chosenBackendIdBySeq(int replicationNum, Multimap locReq, - TStorageMedium storageMedium) - throws DdlException { - List chosenBackendIds = - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().getNodeSelector() - .seqChooseBackendIdsByStorageMedium(replicationNum, - true, true, locReq, storageMedium); - if (CollectionUtils.isEmpty(chosenBackendIds)) { - throw new DdlException( - "Failed to find enough hosts with storage medium " + storageMedium + - " at all backends, number of replicas needed: " + - replicationNum + ". Storage medium check failure can be forcefully ignored by executing " + - "'ADMIN SET FRONTEND CONFIG (\"enable_strict_storage_medium_check\" = \"false\");', " + - "but incompatible medium type can cause balance problem, so we strongly recommend" + - " creating table with compatible 'storage_medium' property set."); - } - return chosenBackendIds; - } - - private List chosenBackendIdBySeq(int replicationNum, Multimap locReq) throws DdlException { - SystemInfoService systemInfoService = GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo(); - List chosenBackendIds = systemInfoService.getNodeSelector() - .seqChooseBackendIds(replicationNum, true, true, locReq); - if (!CollectionUtils.isEmpty(chosenBackendIds)) { - return chosenBackendIds; - } else if (replicationNum > 1) { - List backendIds = systemInfoService.getBackendIds(true); - throw new DdlException( - String.format("Table replication num should be less than or equal to the number of available BE nodes. " - + "You can change this default by setting the replication_num table properties. " - + "Current alive backend is [%s]. ", Joiner.on(",").join(backendIds))); - } else { - throw new DdlException("No alive nodes"); - } - } - - // Drop table - public void dropTable(DropTableStmt stmt) throws DdlException { - String dbName = stmt.getDbName(); - String tableName = stmt.getTableName(); - - // check database - Database db = getDb(dbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - if (db.isSystemDatabase()) { - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "cannot drop table in system database: " + db.getOriginName()); - } - db.dropTable(tableName, stmt.isSetIfExists(), stmt.isForceDrop()); - } - - public void dropTemporaryTable(String dbName, long tableId, String tableName, - boolean isSetIfExsists, boolean isForce) throws DdlException { - Database db = getDb(dbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - db.dropTemporaryTable(tableId, tableName, isSetIfExsists, isForce); - } - - public void replayDropTable(Database db, long tableId, boolean isForceDrop) { - Table table; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - table = getTable(db.getId(), tableId); - if (table.isTemporaryTable()) { - table = db.unprotectDropTemporaryTable(tableId, isForceDrop, false); - UUID sessionId = ((OlapTable) table).getSessionId(); - TemporaryTableMgr temporaryTableMgr = GlobalStateMgr.getCurrentState().getTemporaryTableMgr(); - temporaryTableMgr.dropTemporaryTable(sessionId, db.getId(), table.getName()); - } else { - table = db.unprotectDropTable(tableId, isForceDrop, true); - } - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - if (table != null && isForceDrop) { - table.delete(db.getId(), true); - } - } - - public void replayEraseTable(long tableId) { - recycleBin.replayEraseTable(Collections.singletonList(tableId)); - } - - public void replayEraseMultiTables(MultiEraseTableInfo multiEraseTableInfo) { - List tableIds = multiEraseTableInfo.getTableIds(); - recycleBin.replayEraseTable(tableIds); - } - - public void replayDisableTableRecovery(DisableTableRecoveryInfo disableTableRecoveryInfo) { - recycleBin.replayDisableTableRecovery(disableTableRecoveryInfo.getTableIds()); - } - - public void replayDisablePartitionRecovery(DisablePartitionRecoveryInfo disablePartitionRecoveryInfo) { - recycleBin.replayDisablePartitionRecovery(disablePartitionRecoveryInfo.getPartitionId()); - } - - public void replayRecoverTable(RecoverInfo info) { - long dbId = info.getDbId(); - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - recycleBin.replayRecoverTable(db, info.getTableId()); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayAddReplica(ReplicaPersistInfo info) { - Database db = getDbIncludeRecycleBin(info.getDbId()); - if (db == null) { - LOG.warn("replay add replica failed, db is null, info: {}", info); - return; - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTableIncludeRecycleBin(db, info.getTableId()); - if (olapTable == null) { - LOG.warn("replay add replica failed, table is null, info: {}", info); - return; - } - PhysicalPartition partition = getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); - if (partition == null) { - LOG.warn("replay add replica failed, partition is null, info: {}", info); - return; - } - MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); - if (materializedIndex == null) { - LOG.warn("replay add replica failed, materializedIndex is null, info: {}", info); - return; - } - LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); - if (tablet == null) { - LOG.warn("replay add replica failed, tablet is null, info: {}", info); - return; - } - - // for compatibility - int schemaHash = info.getSchemaHash(); - if (schemaHash == -1) { - schemaHash = olapTable.getSchemaHashByIndexId(info.getIndexId()); - } - - Replica replica = new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), - schemaHash, info.getDataSize(), info.getRowCount(), - Replica.ReplicaState.NORMAL, - info.getLastFailedVersion(), - info.getLastSuccessVersion(), - info.getMinReadableVersion()); - tablet.addReplica(replica); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayUpdateReplica(ReplicaPersistInfo info) { - Database db = getDbIncludeRecycleBin(info.getDbId()); - if (db == null) { - LOG.warn("replay update replica failed, db is null, info: {}", info); - return; - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTableIncludeRecycleBin(db, info.getTableId()); - if (olapTable == null) { - LOG.warn("replay update replica failed, table is null, info: {}", info); - return; - } - PhysicalPartition partition = getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); - if (partition == null) { - LOG.warn("replay update replica failed, partition is null, info: {}", info); - return; - } - MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); - if (materializedIndex == null) { - LOG.warn("replay update replica failed, materializedIndex is null, info: {}", info); - return; - } - LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); - if (tablet == null) { - LOG.warn("replay update replica failed, tablet is null, info: {}", info); - return; - } - Replica replica = tablet.getReplicaByBackendId(info.getBackendId()); - if (replica == null) { - LOG.warn("replay update replica failed, replica is null, info: {}", info); - return; - } - replica.updateRowCount(info.getVersion(), info.getMinReadableVersion(), info.getDataSize(), info.getRowCount()); - replica.setBad(false); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayDeleteReplica(ReplicaPersistInfo info) { - Database db = getDbIncludeRecycleBin(info.getDbId()); - if (db == null) { - LOG.warn("replay delete replica failed, db is null, info: {}", info); - return; - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTableIncludeRecycleBin(db, info.getTableId()); - if (olapTable == null) { - LOG.warn("replay delete replica failed, table is null, info: {}", info); - return; - } - PhysicalPartition partition = getPhysicalPartitionIncludeRecycleBin(olapTable, info.getPartitionId()); - if (partition == null) { - LOG.warn("replay delete replica failed, partition is null, info: {}", info); - return; - } - MaterializedIndex materializedIndex = partition.getIndex(info.getIndexId()); - if (materializedIndex == null) { - LOG.warn("replay delete replica failed, materializedIndex is null, info: {}", info); - return; - } - LocalTablet tablet = (LocalTablet) materializedIndex.getTablet(info.getTabletId()); - if (tablet == null) { - LOG.warn("replay delete replica failed, tablet is null, info: {}", info); - return; - } - tablet.deleteReplicaByBackendId(info.getBackendId()); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayBatchDeleteReplica(BatchDeleteReplicaInfo info) { - if (info.getReplicaInfoList() != null) { - for (ReplicaPersistInfo persistInfo : info.getReplicaInfoList()) { - replayDeleteReplica(persistInfo); - } - } else { - LOG.warn("invalid BatchDeleteReplicaInfo, replicaInfoList is null"); - } - } - - @Override - public Database getDb(String name) { - if (name == null) { - return null; - } - if (fullNameToDb.containsKey(name)) { - return fullNameToDb.get(name); - } else { - // This maybe an 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) || - dbName.equalsIgnoreCase(SysDb.DATABASE_NAME)) { - return fullNameToDb.get(dbName.toLowerCase()); - } - } - return null; - } - - @Override - public Database getDb(long dbId) { - return idToDb.get(dbId); - } - - public Optional mayGetDb(String name) { - return Optional.ofNullable(getDb(name)); - } - - public Optional mayGetDb(long dbId) { - return Optional.ofNullable(getDb(dbId)); - } - - public Optional
mayGetTable(long dbId, long tableId) { - return mayGetDb(dbId).flatMap(db -> Optional.ofNullable(db.getTable(tableId))); - } - - public Optional
mayGetTable(String dbName, String tableName) { - return mayGetDb(dbName).flatMap(db -> Optional.ofNullable(db.getTable(tableName))); - } - - public ConcurrentHashMap getFullNameToDb() { - return fullNameToDb; - } - - public Database getDbIncludeRecycleBin(long dbId) { - Database db = idToDb.get(dbId); - if (db == null) { - db = recycleBin.getDatabase(dbId); - } - return db; - } - - @Override - public boolean tableExists(String dbName, String tblName) { - Database database = getDb(dbName); - if (database == null) { - return false; - } - return database.getTable(tblName) != null; - } - - @Override - public Table getTable(String dbName, String tblName) { - Database database = getDb(dbName); - if (database == null) { - return null; - } - return database.getTable(tblName); - } - - public Table getTable(Long dbId, Long tableId) { - Database database = getDb(dbId); - if (database == null) { - return null; - } - return database.getTable(tableId); - } - - public List
getTables(Long dbId) { - Database database = getDb(dbId); - if (database == null) { - return Collections.emptyList(); - } else { - return database.getTables(); - } - } - - @Override - public Pair getMaterializedViewIndex(String dbName, String indexName) { - Database database = getDb(dbName); - if (database == null) { - return null; - } - return database.getMaterializedViewIndex(indexName); - } - - public Table getTableIncludeRecycleBin(Database db, long tableId) { - Table table = getTable(db.getId(), tableId); - if (table == null) { - table = recycleBin.getTable(db.getId(), tableId); - } - return table; - } - - public List
getTablesIncludeRecycleBin(Database db) { - List
tables = db.getTables(); - tables.addAll(recycleBin.getTables(db.getId())); - return tables; - } - - public Partition getPartitionIncludeRecycleBin(OlapTable table, long partitionId) { - Partition partition = table.getPartition(partitionId); - if (partition == null) { - partition = recycleBin.getPartition(partitionId); - } - return partition; - } - - public PhysicalPartition getPhysicalPartitionIncludeRecycleBin(OlapTable table, long physicalPartitionId) { - PhysicalPartition partition = table.getPhysicalPartition(physicalPartitionId); - if (partition == null) { - partition = recycleBin.getPhysicalPartition(physicalPartitionId); - } - return partition; - } - - public Collection getPartitionsIncludeRecycleBin(OlapTable table) { - Collection partitions = new ArrayList<>(table.getPartitions()); - partitions.addAll(recycleBin.getPartitions(table.getId())); - return partitions; - } - - public Collection getAllPartitionsIncludeRecycleBin(OlapTable table) { - Collection partitions = table.getAllPartitions(); - partitions.addAll(recycleBin.getPartitions(table.getId())); - return partitions; - } - - // NOTE: result can be null, cause partition erase is not in db lock - public DataProperty getDataPropertyIncludeRecycleBin(PartitionInfo info, long partitionId) { - DataProperty dataProperty = info.getDataProperty(partitionId); - if (dataProperty == null) { - dataProperty = recycleBin.getPartitionDataProperty(partitionId); - } - return dataProperty; - } - - // NOTE: result can be -1, cause partition erase is not in db lock - public short getReplicationNumIncludeRecycleBin(PartitionInfo info, long partitionId) { - short replicaNum = info.getReplicationNum(partitionId); - if (replicaNum == (short) -1) { - replicaNum = recycleBin.getPartitionReplicationNum(partitionId); - } - return replicaNum; - } - - @Override - public List listDbNames() { - return Lists.newArrayList(fullNameToDb.keySet()); - } - - @Override - public List listTableNames(String dbName) { - Database database = getDb(dbName); - if (database != null) { - return database.getTables().stream() - .map(Table::getName).collect(Collectors.toList()); - } else { - throw new StarRocksConnectorException("Database " + dbName + " doesn't exist"); - } - } - - @Override - public List getDbIds() { - return Lists.newArrayList(idToDb.keySet()); - } - - public List getDbIdsIncludeRecycleBin() { - List dbIds = getDbIds(); - dbIds.addAll(recycleBin.getAllDbIds()); - return dbIds; - } - - public HashMap getPartitionIdToStorageMediumMap() { - HashMap storageMediumMap = new HashMap<>(); - - // record partition which need to change storage medium - // dbId -> (tableId -> partitionId) - HashMap> changedPartitionsMap = new HashMap<>(); - long currentTimeMs = System.currentTimeMillis(); - List dbIds = getDbIds(); - - for (long dbId : dbIds) { - Database db = getDb(dbId); - if (db == null) { - LOG.warn("db {} does not exist while doing backend report", dbId); - continue; - } - - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - for (Table table : db.getTables()) { - if (!table.isOlapTableOrMaterializedView()) { - continue; - } - - long tableId = table.getId(); - OlapTable olapTable = (OlapTable) table; - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - for (Partition partition : olapTable.getAllPartitions()) { - long partitionId = partition.getId(); - DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); - Preconditions.checkNotNull(dataProperty, - partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId); - // only normal state table can migrate. - // PRIMARY_KEYS table does not support local migration. - if (dataProperty.getStorageMedium() == TStorageMedium.SSD - && dataProperty.getCooldownTimeMs() < currentTimeMs - && olapTable.getState() == OlapTable.OlapTableState.NORMAL) { - // expire. change to HDD. - // record and change when holding write lock - Multimap multimap = changedPartitionsMap.get(dbId); - if (multimap == null) { - multimap = HashMultimap.create(); - changedPartitionsMap.put(dbId, multimap); - } - multimap.put(tableId, partitionId); - } else { - storageMediumMap.put(partitionId, dataProperty.getStorageMedium()); - } - } // end for partitions - } // end for tables - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - } // end for dbs - - // handle data property changed - for (Long dbId : changedPartitionsMap.keySet()) { - Database db = getDb(dbId); - if (db == null) { - LOG.warn("db {} does not exist while checking backend storage medium", dbId); - continue; - } - Multimap tableIdToPartitionIds = changedPartitionsMap.get(dbId); - - // use try lock to avoid blocking a long time. - // if block too long, backend report rpc will timeout. - Locker locker = new Locker(); - if (!locker.tryLockDatabase(db.getId(), LockType.WRITE, Database.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { - LOG.warn("try get db {}-{} write lock but failed when checking backend storage medium", - db.getFullName(), dbId); - continue; - } - Preconditions.checkState(locker.isDbWriteLockHeldByCurrentThread(db)); - try { - for (Long tableId : tableIdToPartitionIds.keySet()) { - Table table = getTable(db.getId(), tableId); - if (table == null) { - continue; - } - OlapTable olapTable = (OlapTable) table; - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - - Collection partitionIds = tableIdToPartitionIds.get(tableId); - for (Long partitionId : partitionIds) { - Partition partition = olapTable.getPartition(partitionId); - if (partition == null) { - continue; - } - DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId()); - if (dataProperty.getStorageMedium() == TStorageMedium.SSD - && dataProperty.getCooldownTimeMs() < currentTimeMs) { - // expire. change to HDD. - DataProperty hdd = new DataProperty(TStorageMedium.HDD); - partitionInfo.setDataProperty(partition.getId(), hdd); - storageMediumMap.put(partitionId, TStorageMedium.HDD); - LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD", - dbId, tableId, partitionId); - - // log - ModifyPartitionInfo info = - new ModifyPartitionInfo(db.getId(), olapTable.getId(), - partition.getId(), - hdd, - (short) -1, - partitionInfo.getIsInMemory(partition.getId())); - GlobalStateMgr.getCurrentState().getEditLog().logModifyPartition(info); - } - } // end for partitions - } // end for tables - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } // end for dbs - return storageMediumMap; - } - - /* - * used for handling AlterTableStmt (for client is the ALTER TABLE command). - * including SchemaChangeHandler and RollupHandler - */ - @Override - public void alterTable(ConnectContext context, AlterTableStmt stmt) throws UserException { - AlterJobExecutor alterJobExecutor = new AlterJobExecutor(); - alterJobExecutor.process(stmt, context); - } - - /** - * used for handling AlterViewStmt (the ALTER VIEW command). - */ - @Override - public void alterView(AlterViewStmt stmt) throws UserException { - new AlterJobExecutor().process(stmt, ConnectContext.get()); - } - - @Override - public void createMaterializedView(CreateMaterializedViewStmt stmt) - throws AnalysisException, DdlException { - MaterializedViewHandler materializedViewHandler = - GlobalStateMgr.getCurrentState().getAlterJobMgr().getMaterializedViewHandler(); - String tableName = stmt.getBaseIndexName(); - // check db - String dbName = stmt.getDBName(); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - // check cluster capacity - GlobalStateMgr.getCurrentState().getNodeMgr().getClusterInfo().checkClusterCapacity(); - // check db quota - db.checkQuota(); - - Locker locker = new Locker(); - if (!locker.lockDatabaseAndCheckExist(db, LockType.WRITE)) { - throw new DdlException("create materialized failed. database:" + db.getFullName() + " not exist"); - } - try { - Table table = getTable(db.getFullName(), tableName); - if (table == null) { - throw new DdlException("create materialized failed. table:" + tableName + " not exist"); - } - if (table.isCloudNativeTable()) { - throw new DdlException("Creating synchronous materialized view(rollup) is not supported in " + - "shared data clusters.\nPlease use asynchronous materialized view instead.\n" + - "Refer to https://docs.starrocks.io/en-us/latest/sql-reference/sql-statements" + - "/data-definition/CREATE%20MATERIALIZED%20VIEW#asynchronous-materialized-view for details."); - } - if (!table.isOlapTable()) { - throw new DdlException("Do not support create synchronous materialized view(rollup) on " + - table.getType().name() + " table[" + tableName + "]"); - } - OlapTable olapTable = (OlapTable) table; - if (olapTable.getKeysType() == KeysType.PRIMARY_KEYS) { - throw new DdlException( - "Do not support create materialized view on primary key table[" + tableName + "]"); - } - if (GlobalStateMgr.getCurrentState().getInsertOverwriteJobMgr().hasRunningOverwriteJob(olapTable.getId())) { - throw new DdlException("Table[" + olapTable.getName() + "] is doing insert overwrite job, " + - "please start to create materialized view after insert overwrite"); - } - olapTable.checkStableAndNormal(); - - materializedViewHandler.processCreateMaterializedView(stmt, db, olapTable); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - // TODO(murphy) refactor it into MVManager - @Override - public void createMaterializedView(CreateMaterializedViewStatement stmt) - throws DdlException { - // check mv exists,name must be different from view/mv/table which exists in metadata - String mvName = stmt.getTableName().getTbl(); - String dbName = stmt.getTableName().getDb(); - LOG.debug("Begin create materialized view: {}", mvName); - // check if db exists - Database db = this.getDb(dbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - - // check if table exists in db - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - if (getTable(db.getFullName(), mvName) != null) { - if (stmt.isIfNotExists()) { - LOG.info("Create materialized view [{}] which already exists", mvName); - return; - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, mvName); - } - } - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - // create columns - List baseSchema = stmt.getMvColumnItems(); - validateColumns(baseSchema); - - Map properties = stmt.getProperties(); - if (properties == null) { - properties = Maps.newHashMap(); - } - - // create partition info - PartitionInfo partitionInfo = buildPartitionInfo(stmt); - // create distribution info - DistributionDesc distributionDesc = stmt.getDistributionDesc(); - Preconditions.checkNotNull(distributionDesc); - DistributionInfo baseDistribution = distributionDesc.toDistributionInfo(baseSchema); - // create refresh scheme - MaterializedView.MvRefreshScheme mvRefreshScheme; - RefreshSchemeClause refreshSchemeDesc = stmt.getRefreshSchemeDesc(); - if (refreshSchemeDesc.getType() == MaterializedView.RefreshType.ASYNC) { - mvRefreshScheme = new MaterializedView.MvRefreshScheme(); - AsyncRefreshSchemeDesc asyncRefreshSchemeDesc = (AsyncRefreshSchemeDesc) refreshSchemeDesc; - MaterializedView.AsyncRefreshContext asyncRefreshContext = mvRefreshScheme.getAsyncRefreshContext(); - asyncRefreshContext.setDefineStartTime(asyncRefreshSchemeDesc.isDefineStartTime()); - int randomizeStart = 0; - if (properties.containsKey(PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START)) { - try { - randomizeStart = Integer.parseInt(properties.get((PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START))); - } catch (NumberFormatException e) { - ErrorReport.reportSemanticException(ErrorCode.ERR_INVALID_PARAMETER, - PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START + " only accept integer as parameter"); - } - // remove this transient variable - properties.remove(PropertyAnalyzer.PROPERTY_MV_RANDOMIZE_START); - } - - long random = getRandomStart(asyncRefreshSchemeDesc.getIntervalLiteral(), randomizeStart); - if (asyncRefreshSchemeDesc.isDefineStartTime() || randomizeStart == -1) { - long definedStartTime = Utils.getLongFromDateTime(asyncRefreshSchemeDesc.getStartTime()); - // Add random set only if mv_random_start > 0 when user has already set the start time - if (randomizeStart > 0) { - definedStartTime += random; - } - asyncRefreshContext.setStartTime(definedStartTime); - } else if (asyncRefreshSchemeDesc.getIntervalLiteral() != null) { - long currentTimeSecond = Utils.getLongFromDateTime(LocalDateTime.now()); - long randomizedStart = currentTimeSecond + random; - asyncRefreshContext.setStartTime(randomizedStart); - } - if (asyncRefreshSchemeDesc.getIntervalLiteral() != null) { - long intervalStep = ((IntLiteral) asyncRefreshSchemeDesc.getIntervalLiteral().getValue()).getValue(); - String refreshTimeUnit = asyncRefreshSchemeDesc.getIntervalLiteral().getUnitIdentifier().getDescription(); - asyncRefreshContext.setStep(intervalStep); - asyncRefreshContext.setTimeUnit(refreshTimeUnit); - - // Check the interval time should not be less than the min allowed config time. - if (Config.materialized_view_min_refresh_interval > 0) { - TimeUnit intervalTimeUnit = TimeUtils.convertUnitIdentifierToTimeUnit(refreshTimeUnit); - long periodSeconds = TimeUtils.convertTimeUnitValueToSecond(intervalStep, intervalTimeUnit); - if (periodSeconds < Config.materialized_view_min_refresh_interval) { - throw new DdlException(String.format("Refresh schedule interval %s is too small which may cost " + - "a lot of memory/cpu resources to refresh the asynchronous materialized view, " + - "please config an interval larger than " + - "Config.materialized_view_min_refresh_interval(%ss).", - periodSeconds, - Config.materialized_view_min_refresh_interval)); - } - } - } - - // task which type is EVENT_TRIGGERED can not use external table as base table now. - if (asyncRefreshContext.getTimeUnit() == null) { - // asyncRefreshContext's timeUnit is null means this task's type is EVENT_TRIGGERED - Map tableNameTableMap = AnalyzerUtils.collectAllTable(stmt.getQueryStatement()); - if (tableNameTableMap.values().stream().anyMatch(table -> !table.isNativeTableOrMaterializedView())) { - throw new DdlException( - "Materialized view which type is ASYNC need to specify refresh interval for " + - "external table"); - } - } - } else if (refreshSchemeDesc.getType() == MaterializedView.RefreshType.SYNC) { - mvRefreshScheme = new MaterializedView.MvRefreshScheme(); - mvRefreshScheme.setType(MaterializedView.RefreshType.SYNC); - } else if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.MANUAL)) { - mvRefreshScheme = new MaterializedView.MvRefreshScheme(); - mvRefreshScheme.setType(MaterializedView.RefreshType.MANUAL); - } else { - mvRefreshScheme = new MaterializedView.MvRefreshScheme(); - mvRefreshScheme.setType(MaterializedView.RefreshType.INCREMENTAL); - } - mvRefreshScheme.setMoment(refreshSchemeDesc.getMoment()); - // create mv - long mvId = GlobalStateMgr.getCurrentState().getNextId(); - MaterializedView materializedView; - if (RunMode.isSharedNothingMode()) { - if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.INCREMENTAL)) { - materializedView = GlobalStateMgr.getCurrentState().getMaterializedViewMgr() - .createSinkTable(stmt, partitionInfo, mvId, db.getId()); - materializedView.setMaintenancePlan(stmt.getMaintenancePlan()); - } else { - materializedView = - new MaterializedView(mvId, db.getId(), mvName, baseSchema, stmt.getKeysType(), partitionInfo, - baseDistribution, mvRefreshScheme); - } - } else { - Preconditions.checkState(RunMode.isSharedDataMode()); - if (refreshSchemeDesc.getType().equals(MaterializedView.RefreshType.INCREMENTAL)) { - throw new DdlException("Incremental materialized view in shared_data mode is not supported"); - } - - materializedView = - new LakeMaterializedView(mvId, db.getId(), mvName, baseSchema, stmt.getKeysType(), partitionInfo, - baseDistribution, mvRefreshScheme); - } - - //bitmap indexes - List mvIndexes = stmt.getMvIndexes(); - materializedView.setIndexes(mvIndexes); - - // sort keys - if (CollectionUtils.isNotEmpty(stmt.getSortKeys())) { - materializedView.setTableProperty(new TableProperty()); - materializedView.getTableProperty().setMvSortKeys(stmt.getSortKeys()); - } - // set comment - materializedView.setComment(stmt.getComment()); - // set baseTableIds - materializedView.setBaseTableInfos(stmt.getBaseTableInfos()); - // set viewDefineSql - materializedView.setViewDefineSql(stmt.getInlineViewDef()); - materializedView.setSimpleDefineSql(stmt.getSimpleViewDef()); - materializedView.setOriginalViewDefineSql(stmt.getOriginalViewDefineSql()); - // set partitionRefTableExprs - if (stmt.getPartitionRefTableExpr() != null) { - //avoid to get a list of null inside - materializedView.setPartitionRefTableExprs(Lists.newArrayList(stmt.getPartitionRefTableExpr())); - } - // set base index id - long baseIndexId = getNextId(); - materializedView.setBaseIndexId(baseIndexId); - // set query output indexes - materializedView.setQueryOutputIndices(stmt.getQueryOutputIndices()); - // set base index meta - int schemaVersion = 0; - int schemaHash = Util.schemaHash(schemaVersion, baseSchema, null, 0d); - short shortKeyColumnCount = GlobalStateMgr.calcShortKeyColumnCount(baseSchema, null); - TStorageType baseIndexStorageType = TStorageType.COLUMN; - materializedView.setIndexMeta(baseIndexId, mvName, baseSchema, schemaVersion, schemaHash, - shortKeyColumnCount, baseIndexStorageType, stmt.getKeysType()); - - // validate hint - Map optHints = Maps.newHashMap(); - if (stmt.isExistQueryScopeHint()) { - SessionVariable sessionVariable = VariableMgr.newSessionVariable(); - for (HintNode hintNode : stmt.getAllQueryScopeHints()) { - if (hintNode instanceof SetVarHint) { - for (Map.Entry entry : hintNode.getValue().entrySet()) { - VariableMgr.setSystemVariable(sessionVariable, - new SystemVariable(entry.getKey(), new StringLiteral(entry.getValue())), true); - optHints.put(entry.getKey(), entry.getValue()); - } - } else if (hintNode instanceof UserVariableHint) { - throw new DdlException("unsupported user variable hint in Materialized view for now."); - } - } - } - - boolean isNonPartitioned = partitionInfo.isUnPartitioned(); - DataProperty dataProperty = PropertyAnalyzer.analyzeMVDataProperty(materializedView, properties); - PropertyAnalyzer.analyzeMVProperties(db, materializedView, properties, isNonPartitioned); - try { - Set tabletIdSet = new HashSet<>(); - // process single partition info - if (isNonPartitioned) { - long partitionId = GlobalStateMgr.getCurrentState().getNextId(); - Preconditions.checkNotNull(dataProperty); - partitionInfo.setDataProperty(partitionId, dataProperty); - partitionInfo.setReplicationNum(partitionId, materializedView.getDefaultReplicationNum()); - partitionInfo.setIsInMemory(partitionId, false); - partitionInfo.setTabletType(partitionId, TTabletType.TABLET_TYPE_DISK); - StorageInfo storageInfo = materializedView.getTableProperty().getStorageInfo(); - partitionInfo.setDataCacheInfo(partitionId, - storageInfo == null ? null : storageInfo.getDataCacheInfo()); - Long version = Partition.PARTITION_INIT_VERSION; - Partition partition = createPartition(db, materializedView, partitionId, mvName, version, tabletIdSet, - materializedView.getWarehouseId()); - buildPartitions(db, materializedView, new ArrayList<>(partition.getSubPartitions()), - materializedView.getWarehouseId()); - materializedView.addPartition(partition); - } else { - Expr partitionExpr = stmt.getPartitionExpDesc().getExpr(); - Map partitionExprMaps = MVPartitionExprResolver.getMVPartitionExprsChecked(partitionExpr, - stmt.getQueryStatement(), stmt.getBaseTableInfos()); - LOG.info("Generate mv {} partition exprs: {}", mvName, partitionExprMaps); - materializedView.setPartitionExprMaps(partitionExprMaps); - } - - GlobalStateMgr.getCurrentState().getMaterializedViewMgr().prepareMaintenanceWork(stmt, materializedView); - - String storageVolumeId = ""; - if (materializedView.isCloudNativeMaterializedView()) { - storageVolumeId = GlobalStateMgr.getCurrentState().getStorageVolumeMgr() - .getStorageVolumeIdOfTable(materializedView.getId()); - } - onCreate(db, materializedView, storageVolumeId, stmt.isIfNotExists()); - } catch (DdlException e) { - if (materializedView.isCloudNativeMaterializedView()) { - GlobalStateMgr.getCurrentState().getStorageVolumeMgr().unbindTableToStorageVolume(materializedView.getId()); - } - throw e; - } - LOG.info("Successfully create materialized view [{}:{}]", mvName, materializedView.getMvId()); - - // NOTE: The materialized view has been added to the database, and the following procedure cannot throw exception. - createTaskForMaterializedView(dbName, materializedView, optHints); - DynamicPartitionUtil.registerOrRemovePartitionTTLTable(db.getId(), materializedView); - } - - private long getRandomStart(IntervalLiteral interval, long randomizeStart) throws DdlException { - if (interval == null || randomizeStart == -1) { - return 0; - } - // randomize the start time if not specified manually, to avoid refresh conflicts - // default random interval is min(300s, INTERVAL/2) - // user could specify it through mv_randomize_start - long period = ((IntLiteral) interval.getValue()).getLongValue(); - TimeUnit timeUnit = - TimeUtils.convertUnitIdentifierToTimeUnit(interval.getUnitIdentifier().getDescription()); - long intervalSeconds = TimeUtils.convertTimeUnitValueToSecond(period, timeUnit); - long randomInterval = randomizeStart == 0 ? Math.min(300, intervalSeconds / 2) : randomizeStart; - return randomInterval > 0 ? ThreadLocalRandom.current().nextLong(randomInterval) : randomInterval; - } - - public static PartitionInfo buildPartitionInfo(CreateMaterializedViewStatement stmt) throws DdlException { - ExpressionPartitionDesc expressionPartitionDesc = stmt.getPartitionExpDesc(); - if (expressionPartitionDesc != null) { - Expr expr = expressionPartitionDesc.getExpr(); - if (expr instanceof SlotRef) { - SlotRef slotRef = (SlotRef) expr; - if (slotRef.getType().getPrimitiveType() == PrimitiveType.VARCHAR) { - return new ListPartitionInfo(PartitionType.LIST, - Collections.singletonList(stmt.getPartitionColumn())); - } - } - if ((expr instanceof FunctionCallExpr)) { - FunctionCallExpr functionCallExpr = (FunctionCallExpr) expr; - if (functionCallExpr.getFnName().getFunction().equalsIgnoreCase(FunctionSet.STR2DATE)) { - Column partitionColumn = new Column(stmt.getPartitionColumn()); - partitionColumn.setType(com.starrocks.catalog.Type.DATE); - return expressionPartitionDesc.toPartitionInfo( - Collections.singletonList(partitionColumn), - Maps.newHashMap(), false); - } - } - return expressionPartitionDesc.toPartitionInfo( - Collections.singletonList(stmt.getPartitionColumn()), - Maps.newHashMap(), false); - } else { - return new SinglePartitionInfo(); - } - } - - private void createTaskForMaterializedView(String dbName, MaterializedView materializedView, - Map optHints) throws DdlException { - MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); - MaterializedView.RefreshMoment refreshMoment = materializedView.getRefreshScheme().getMoment(); - - if (refreshType.equals(MaterializedView.RefreshType.INCREMENTAL)) { - GlobalStateMgr.getCurrentState().getMaterializedViewMgr().startMaintainMV(materializedView); - return; - } - - if (refreshType != MaterializedView.RefreshType.SYNC) { - - Task task = TaskBuilder.buildMvTask(materializedView, dbName); - TaskBuilder.updateTaskInfo(task, materializedView); - - if (optHints != null) { - Map taskProperties = task.getProperties(); - taskProperties.putAll(optHints); - } - - TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); - taskManager.createTask(task, false); - if (refreshMoment.equals(MaterializedView.RefreshMoment.IMMEDIATE)) { - taskManager.executeTask(task.getName()); - } - } - } - - /** - * Leave some clean up work to {@link MaterializedView#onDrop} - */ - @Override - public void dropMaterializedView(DropMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { - Database db = getDb(stmt.getDbName()); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, stmt.getDbName()); - } - Table table; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - table = getTable(db.getFullName(), stmt.getMvName()); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - if (table instanceof MaterializedView) { - try { - Authorizer.checkMaterializedViewAction(ConnectContext.get().getCurrentUserIdentity(), - ConnectContext.get().getCurrentRoleIds(), stmt.getDbMvName(), PrivilegeType.DROP); - } catch (AccessDeniedException e) { - AccessDeniedException.reportAccessDenied( - stmt.getDbMvName().getCatalog(), - ConnectContext.get().getCurrentUserIdentity(), - ConnectContext.get().getCurrentRoleIds(), PrivilegeType.DROP.name(), ObjectType.MATERIALIZED_VIEW.name(), - stmt.getDbMvName().getTbl()); - } - - db.dropTable(table.getName(), stmt.isSetIfExists(), true); - } else { - stateMgr.getAlterJobMgr().processDropMaterializedView(stmt); - } - } - - @Override - public void alterMaterializedView(AlterMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { - new AlterMVJobExecutor().process(stmt, ConnectContext.get()); - } - - private String executeRefreshMvTask(String dbName, MaterializedView materializedView, - ExecuteOption executeOption) - throws DdlException { - MaterializedView.RefreshType refreshType = materializedView.getRefreshScheme().getType(); - LOG.info("Start to execute refresh materialized view task, mv: {}, refreshType: {}, executionOption:{}", - materializedView.getName(), refreshType, executeOption); - - if (refreshType.equals(MaterializedView.RefreshType.INCREMENTAL)) { - GlobalStateMgr.getCurrentState().getMaterializedViewMgr().onTxnPublish(materializedView); - } else if (refreshType != MaterializedView.RefreshType.SYNC) { - TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); - final String mvTaskName = TaskBuilder.getMvTaskName(materializedView.getId()); - if (!taskManager.containTask(mvTaskName)) { - Task task = TaskBuilder.buildMvTask(materializedView, dbName); - TaskBuilder.updateTaskInfo(task, materializedView); - taskManager.createTask(task, false); - } - return taskManager.executeTask(mvTaskName, executeOption).getQueryId(); - } - return null; - } - - private MaterializedView getMaterializedViewToRefresh(String dbName, String mvName) - throws DdlException, MetaNotFoundException { - Database db = this.getDb(dbName); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbName); - } - final Table table = getTable(db.getFullName(), mvName); - MaterializedView materializedView = null; - if (table instanceof MaterializedView) { - materializedView = (MaterializedView) table; - } - if (materializedView == null) { - throw new MetaNotFoundException(mvName + " is not a materialized view"); - } - return materializedView; - } - - public String refreshMaterializedView(String dbName, String mvName, boolean force, PartitionRangeDesc range, - int priority, boolean mergeRedundant, boolean isManual) - throws DdlException, MetaNotFoundException { - return refreshMaterializedView(dbName, mvName, force, range, priority, mergeRedundant, isManual, false); - } - - public String refreshMaterializedView(String dbName, String mvName, boolean force, PartitionRangeDesc range, - int priority, boolean mergeRedundant, boolean isManual, boolean isSync) - throws DdlException, MetaNotFoundException { - MaterializedView materializedView = getMaterializedViewToRefresh(dbName, mvName); - - HashMap taskRunProperties = new HashMap<>(); - taskRunProperties.put(TaskRun.PARTITION_START, range == null ? null : range.getPartitionStart()); - taskRunProperties.put(TaskRun.PARTITION_END, range == null ? null : range.getPartitionEnd()); - taskRunProperties.put(TaskRun.FORCE, Boolean.toString(force)); - - ExecuteOption executeOption = new ExecuteOption(priority, mergeRedundant, taskRunProperties); - executeOption.setManual(isManual); - executeOption.setSync(isSync); - return executeRefreshMvTask(dbName, materializedView, executeOption); - } - - @Override - public String refreshMaterializedView(RefreshMaterializedViewStatement refreshMaterializedViewStatement) - throws DdlException, MetaNotFoundException { - String dbName = refreshMaterializedViewStatement.getMvName().getDb(); - String mvName = refreshMaterializedViewStatement.getMvName().getTbl(); - boolean force = refreshMaterializedViewStatement.isForceRefresh(); - PartitionRangeDesc range = refreshMaterializedViewStatement.getPartitionRangeDesc(); - return refreshMaterializedView(dbName, mvName, force, range, Constants.TaskRunPriority.HIGH.value(), - Config.enable_mv_refresh_sync_refresh_mergeable, true, refreshMaterializedViewStatement.isSync()); - } - - @Override - public void cancelRefreshMaterializedView( - CancelRefreshMaterializedViewStmt stmt) throws DdlException, MetaNotFoundException { - String dbName = stmt.getMvName().getDb(); - String mvName = stmt.getMvName().getTbl(); - MaterializedView materializedView = getMaterializedViewToRefresh(dbName, mvName); - TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); - Task refreshTask = taskManager.getTask(TaskBuilder.getMvTaskName(materializedView.getId())); - boolean isForce = stmt.isForce(); - if (refreshTask != null) { - taskManager.killTask(refreshTask.getName(), isForce); - } - } - - /* - * used for handling CacnelAlterStmt (for client is the CANCEL ALTER - * command). including SchemaChangeHandler and RollupHandler - */ - public void cancelAlter(CancelAlterTableStmt stmt, String reason) throws DdlException { - if (stmt.getAlterType() == ShowAlterStmt.AlterType.ROLLUP) { - stateMgr.getRollupHandler().cancel(stmt, reason); - } else if (stmt.getAlterType() == ShowAlterStmt.AlterType.COLUMN - || stmt.getAlterType() == ShowAlterStmt.AlterType.OPTIMIZE) { - stateMgr.getSchemaChangeHandler().cancel(stmt, reason); - } else if (stmt.getAlterType() == ShowAlterStmt.AlterType.MATERIALIZED_VIEW) { - stateMgr.getRollupHandler().cancelMV(stmt); - } else { - throw new DdlException("Cancel " + stmt.getAlterType() + " does not implement yet"); - } - } - - public void cancelAlter(CancelAlterTableStmt stmt) throws DdlException { - cancelAlter(stmt, "user cancelled"); - } - - // entry of rename table operation - @Override - public void renameTable(Database db, Table table, TableRenameClause tableRenameClause) throws DdlException { - OlapTable olapTable = (OlapTable) table; - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); - } - - String oldTableName = olapTable.getName(); - String newTableName = tableRenameClause.getNewTableName(); - if (oldTableName.equals(newTableName)) { - throw new DdlException("Same table name"); - } - - // check if name is already used - if (getTable(db.getFullName(), newTableName) != null) { - throw new DdlException("Table name[" + newTableName + "] is already used"); - } - - olapTable.checkAndSetName(newTableName, false); - - db.dropTable(oldTableName); - db.registerTableUnlocked(olapTable); - inactiveRelatedMaterializedView(db, olapTable, - MaterializedViewExceptions.inactiveReasonForBaseTableRenamed(oldTableName)); - - TableInfo tableInfo = TableInfo.createForTableRename(db.getId(), olapTable.getId(), newTableName); - GlobalStateMgr.getCurrentState().getEditLog().logTableRename(tableInfo); - LOG.info("rename table[{}] to {}, tableId: {}", oldTableName, newTableName, olapTable.getId()); - } - - @Override - public void alterTableComment(Database db, Table table, AlterTableCommentClause clause) { - ModifyTablePropertyOperationLog log = new ModifyTablePropertyOperationLog(db.getId(), table.getId()); - log.setComment(clause.getNewComment()); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(log); - - table.setComment(clause.getNewComment()); - } - - public static void inactiveRelatedMaterializedView(Database db, Table olapTable, String reason) { - for (MvId mvId : olapTable.getRelatedMaterializedViews()) { - MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore() - .getTable(db.getId(), mvId.getId()); - if (mv != null) { - LOG.warn("Inactive MV {}/{} because {}", mv.getName(), mv.getId(), reason); - mv.setInactiveAndReason(reason); - - // recursive inactive - inactiveRelatedMaterializedView(db, mv, - MaterializedViewExceptions.inactiveReasonForBaseTableActive(mv.getName())); - } else { - LOG.info("Ignore materialized view {} does not exists", mvId); - } - } - } - - public void replayRenameTable(TableInfo tableInfo) { - long dbId = tableInfo.getDbId(); - long tableId = tableInfo.getTableId(); - String newTableName = tableInfo.getNewTableName(); - - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable table = (OlapTable) getTable(db.getId(), tableId); - String tableName = table.getName(); - db.dropTable(tableName); - table.setName(newTableName); - db.registerTableUnlocked(table); - inactiveRelatedMaterializedView(db, table, - MaterializedViewExceptions.inactiveReasonForBaseTableRenamed(tableName)); - - LOG.info("replay rename table[{}] to {}, tableId: {}", tableName, newTableName, table.getId()); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - @Override - public void renamePartition(Database db, Table table, PartitionRenameClause renameClause) throws DdlException { - OlapTable olapTable = (OlapTable) table; - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw new DdlException("Table[" + olapTable.getName() + "] is under " + olapTable.getState()); - } - - if (!olapTable.getPartitionInfo().isRangePartition()) { - throw new DdlException("Table[" + olapTable.getName() + "] is single partitioned. " - + "no need to rename partition name."); - } - - String partitionName = renameClause.getPartitionName(); - String newPartitionName = renameClause.getNewPartitionName(); - if (partitionName.equalsIgnoreCase(newPartitionName)) { - throw new DdlException("Same partition name"); - } - - Partition partition = olapTable.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition[" + partitionName + "] does not exists"); - } - - // check if name is already used - if (olapTable.checkPartitionNameExist(newPartitionName)) { - throw new DdlException("Partition name[" + newPartitionName + "] is already used"); - } - - olapTable.renamePartition(partitionName, newPartitionName); - - // log - TableInfo tableInfo = TableInfo.createForPartitionRename(db.getId(), olapTable.getId(), partition.getId(), - newPartitionName); - GlobalStateMgr.getCurrentState().getEditLog().logPartitionRename(tableInfo); - LOG.info("rename partition[{}] to {}", partitionName, newPartitionName); - } - - public void replayRenamePartition(TableInfo tableInfo) { - long dbId = tableInfo.getDbId(); - long tableId = tableInfo.getTableId(); - long partitionId = tableInfo.getPartitionId(); - String newPartitionName = tableInfo.getNewPartitionName(); - - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable table = (OlapTable) getTable(db.getId(), tableId); - Partition partition = table.getPartition(partitionId); - table.renamePartition(partition.getName(), newPartitionName); - LOG.info("replay rename partition[{}] to {}", partition.getName(), newPartitionName); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void renameRollup(Database db, OlapTable table, RollupRenameClause renameClause) throws DdlException { - if (table.getState() != OlapTable.OlapTableState.NORMAL) { - throw new DdlException("Table[" + table.getName() + "] is under " + table.getState()); - } - - String rollupName = renameClause.getRollupName(); - // check if it is base table name - if (rollupName.equals(table.getName())) { - throw new DdlException("Using ALTER TABLE RENAME to change table name"); - } - - String newRollupName = renameClause.getNewRollupName(); - if (rollupName.equals(newRollupName)) { - throw new DdlException("Same rollup name"); - } - - Map indexNameToIdMap = table.getIndexNameToId(); - if (indexNameToIdMap.get(rollupName) == null) { - throw new DdlException("Rollup index[" + rollupName + "] does not exists"); - } - - // check if name is already used - if (indexNameToIdMap.get(newRollupName) != null) { - throw new DdlException("Rollup name[" + newRollupName + "] is already used"); - } - - long indexId = indexNameToIdMap.remove(rollupName); - indexNameToIdMap.put(newRollupName, indexId); - - // log - TableInfo tableInfo = TableInfo.createForRollupRename(db.getId(), table.getId(), indexId, newRollupName); - GlobalStateMgr.getCurrentState().getEditLog().logRollupRename(tableInfo); - LOG.info("rename rollup[{}] to {}", rollupName, newRollupName); - } - - public void replayRenameRollup(TableInfo tableInfo) { - long dbId = tableInfo.getDbId(); - long tableId = tableInfo.getTableId(); - long indexId = tableInfo.getIndexId(); - String newRollupName = tableInfo.getNewRollupName(); - - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable table = (OlapTable) getTable(db.getId(), tableId); - String rollupName = table.getIndexNameById(indexId); - Map indexNameToIdMap = table.getIndexNameToId(); - indexNameToIdMap.remove(rollupName); - indexNameToIdMap.put(newRollupName, indexId); - - LOG.info("replay rename rollup[{}] to {}", rollupName, newRollupName); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void renameColumn(Database db, Table table, ColumnRenameClause renameClause) { - if (!(table instanceof OlapTable)) { - throw ErrorReportException.report(ErrorCode.ERR_COLUMN_RENAME_ONLY_FOR_OLAP_TABLE); - } - if (db.isSystemDatabase() || db.isStatisticsDatabase()) { - throw ErrorReportException.report(ErrorCode.ERR_CANNOT_RENAME_COLUMN_IN_INTERNAL_DB, db.getFullName()); - } - OlapTable olapTable = (OlapTable) table; - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw ErrorReportException.report(ErrorCode.ERR_CANNOT_RENAME_COLUMN_OF_NOT_NORMAL_TABLE, olapTable.getState()); - } - - String colName = renameClause.getColName(); - String newColName = renameClause.getNewColName(); - - Column column = olapTable.getColumn(colName); - if (column == null) { - throw ErrorReportException.report(ErrorCode.ERR_BAD_FIELD_ERROR, colName, table.getName()); - } - Column currentColumn = olapTable.getColumn(newColName); - if (currentColumn != null) { - throw ErrorReportException.report(ErrorCode.ERR_DUP_FIELDNAME, newColName); - } - olapTable.renameColumn(colName, newColName); - - ColumnRenameInfo columnRenameInfo = new ColumnRenameInfo(db.getId(), table.getId(), colName, newColName); - GlobalStateMgr.getCurrentState().getEditLog().logColumnRename(columnRenameInfo); - LOG.info("rename column {} to {}", colName, newColName); - } - - public void replayRenameColumn(ColumnRenameInfo columnRenameInfo) throws DdlException { - long dbId = columnRenameInfo.getDbId(); - long tableId = columnRenameInfo.getTableId(); - String colName = columnRenameInfo.getColumnName(); - String newColName = columnRenameInfo.getNewColumnName(); - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), tableId); - olapTable.renameColumn(colName, newColName); - LOG.info("replay rename column[{}] to {}", colName, newColName); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void modifyTableDynamicPartition(Database db, OlapTable table, Map properties) - throws DdlException { - Map logProperties = new HashMap<>(properties); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - DynamicPartitionUtil.checkAndSetDynamicPartitionProperty(table, properties); - } else { - Map analyzedDynamicPartition = DynamicPartitionUtil.analyzeDynamicPartition(properties); - tableProperty.modifyTableProperties(analyzedDynamicPartition); - tableProperty.buildDynamicProperty(); - } - - DynamicPartitionUtil.registerOrRemovePartitionScheduleInfo(db.getId(), table); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), logProperties); - GlobalStateMgr.getCurrentState().getEditLog().logDynamicPartition(info); - } - - public void alterTableProperties(Database db, OlapTable table, Map properties) - throws DdlException { - Map propertiesToPersist = new HashMap<>(properties); - Map results = validateToBeModifiedProps(properties, table); - - TableProperty tableProperty = table.getTableProperty(); - for (String key : results.keySet()) { - if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER)) { - int partitionLiveNumber = (int) results.get(key); - tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER, - String.valueOf(partitionLiveNumber)); - if (partitionLiveNumber == TableProperty.INVALID) { - GlobalStateMgr.getCurrentState().getDynamicPartitionScheduler().removeTtlPartitionTable(db.getId(), - table.getId()); - } else { - GlobalStateMgr.getCurrentState().getDynamicPartitionScheduler().registerTtlPartitionTable(db.getId(), - table.getId()); - } - tableProperty.setPartitionTTLNumber(partitionLiveNumber); - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - ImmutableMap.of(key, propertiesToPersist.get(key))); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); - } - if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM)) { - DataProperty dataProperty = (DataProperty) results.get(key); - TStorageMedium storageMedium = dataProperty.getStorageMedium(); - table.setStorageMedium(storageMedium); - tableProperty.getProperties() - .put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TIME, - String.valueOf(dataProperty.getCooldownTimeMs())); - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - ImmutableMap.of(key, propertiesToPersist.get(key))); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); - } - if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL)) { - String storageCoolDownTTL = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL); - tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL, storageCoolDownTTL); - tableProperty.buildStorageCoolDownTTL(); - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - ImmutableMap.of(key, propertiesToPersist.get(key))); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); - } - if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION)) { - String partitionDuration = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION); - tableProperty.getProperties().put(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION, partitionDuration); - tableProperty.buildDataCachePartitionDuration(); - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - ImmutableMap.of(key, propertiesToPersist.get(key))); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); - } - if (propertiesToPersist.containsKey(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION)) { - String location = propertiesToPersist.get(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION); - table.setLocation(location); - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - ImmutableMap.of(key, propertiesToPersist.get(key))); - GlobalStateMgr.getCurrentState().getEditLog().logAlterTableProperties(info); - } - } - } - - private Map validateToBeModifiedProps(Map properties, OlapTable table) throws DdlException { - Map results = Maps.newHashMap(); - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER)) { - int partitionLiveNumber = PropertyAnalyzer.analyzePartitionLiveNumber(properties, true); - results.put(PropertyAnalyzer.PROPERTIES_PARTITION_LIVE_NUMBER, partitionLiveNumber); - } - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM)) { - try { - DataProperty dataProperty = DataProperty.getInferredDefaultDataProperty(); - dataProperty = PropertyAnalyzer.analyzeDataProperty(properties, dataProperty, false); - results.put(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM, dataProperty); - } catch (AnalysisException ex) { - throw new RuntimeException(ex.getMessage()); - } - } - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL)) { - try { - PropertyAnalyzer.analyzeStorageCoolDownTTL(properties, true); - results.put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TTL, null); - } catch (AnalysisException ex) { - throw new RuntimeException(ex.getMessage()); - } - } - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION)) { - try { - PropertyAnalyzer.analyzeDataCachePartitionDuration(properties); - results.put(PropertyAnalyzer.PROPERTIES_DATACACHE_PARTITION_DURATION, null); - } catch (AnalysisException ex) { - throw new RuntimeException(ex.getMessage()); - } - } - if (properties.containsKey(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION)) { - if (table.getColocateGroup() != null) { - throw new DdlException("Cannot set location for colocate table"); - } - String locations = PropertyAnalyzer.analyzeLocation(properties, true); - results.put(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION, locations); - } - if (!properties.isEmpty()) { - throw new DdlException("Modify failed because unknown properties: " + properties); - } - return results; - } - - /** - * Set replication number for unpartitioned table. - * ATTN: only for unpartitioned table now. - * - * @param db - * @param table - * @param properties - * @throws DdlException - */ - // The caller need to hold the db write lock - public void modifyTableReplicationNum(Database db, OlapTable table, Map properties) - throws DdlException { - if (colocateTableIndex.isColocateTable(table.getId())) { - throw new DdlException("table " + table.getName() + " is colocate table, cannot change replicationNum"); - } - - String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM; - PartitionInfo partitionInfo = table.getPartitionInfo(); - if (partitionInfo.isRangePartition()) { - throw new DdlException( - "This is a range partitioned table, you should specify partitions with MODIFY PARTITION clause." + - " If you want to set default replication number, please use '" + defaultReplicationNumName + - "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM + "' to escape misleading."); - } - - // unpartitioned table - // update partition replication num - String partitionName = table.getName(); - Partition partition = table.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition does not exist. name: " + partitionName); - } - - short replicationNum = Short.parseShort(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); - boolean isInMemory = partitionInfo.getIsInMemory(partition.getId()); - DataProperty newDataProperty = partitionInfo.getDataProperty(partition.getId()); - partitionInfo.setReplicationNum(partition.getId(), replicationNum); - - // update table default replication num - table.setReplicationNum(replicationNum); - - // log - ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(), - newDataProperty, replicationNum, isInMemory); - GlobalStateMgr.getCurrentState().getEditLog().logModifyPartition(info); - LOG.info("modify partition[{}-{}-{}] replication num to {}", db.getOriginName(), table.getName(), - partition.getName(), replicationNum); - } - - /** - * Set default replication number for a specified table. - * You can see the default replication number by Show Create Table stmt. - * - * @param db - * @param table - * @param properties - */ - // The caller need to hold the db write lock - public void modifyTableDefaultReplicationNum(Database db, OlapTable table, Map properties) - throws DdlException { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - if (colocateTableIndex.isColocateTable(table.getId())) { - throw new DdlException("table " + table.getName() + " is colocate table, cannot change replicationNum"); - } - - // check unpartitioned table - PartitionInfo partitionInfo = table.getPartitionInfo(); - Partition partition = null; - boolean isUnpartitionedTable = false; - if (partitionInfo.getType() == PartitionType.UNPARTITIONED) { - isUnpartitionedTable = true; - String partitionName = table.getName(); - partition = table.getPartition(partitionName); - if (partition == null) { - throw new DdlException("Partition does not exist. name: " + partitionName); - } - } - - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildReplicationNum(); - - // update partition replication num if this table is unpartitioned table - if (isUnpartitionedTable) { - Preconditions.checkNotNull(partition); - partitionInfo.setReplicationNum(partition.getId(), tableProperty.getReplicationNum()); - } - - // log - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyReplicationNum(info); - LOG.info("modify table[{}] replication num to {}", table.getName(), - properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); - } - - public void modifyTableEnablePersistentIndexMeta(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildEnablePersistentIndex(); - - if (table.isCloudNativeTable()) { - // now default to LOCAL - tableProperty.buildPersistentIndexType(); - } - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyEnablePersistentIndex(info); - - } - - public void modifyBinlogMeta(Database db, OlapTable table, BinlogConfig binlogConfig) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - ModifyTablePropertyOperationLog log = new ModifyTablePropertyOperationLog( - db.getId(), - table.getId(), - binlogConfig.toProperties()); - GlobalStateMgr.getCurrentState().getEditLog().logModifyBinlogConfig(log); - - if (!binlogConfig.getBinlogEnable()) { - table.clearBinlogAvailableVersion(); - table.setBinlogTxnId(BinlogConfig.INVALID); - } - table.setCurBinlogConfig(binlogConfig); - } - - // The caller need to hold the db write lock - public void modifyTableInMemoryMeta(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildInMemory(); - - // need to update partition info meta - for (Partition partition : table.getPartitions()) { - table.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory()); - } - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyInMemory(info); - } - - // The caller need to hold the db write lock - public void modifyTableConstraint(Database db, String tableName, Map properties) - throws DdlException { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - Table table = getTable(db.getFullName(), tableName); - if (table == null) { - throw new DdlException(String.format("table:%s does not exist", tableName)); - } - OlapTable olapTable = (OlapTable) table; - TableProperty tableProperty = olapTable.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - olapTable.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildConstraint(); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), olapTable.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyConstraint(info); - } - - // The caller need to hold the db write lock - public void modifyTableWriteQuorum(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildWriteQuorum(); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyWriteQuorum(info); - } - - // The caller need to hold the db write lock - public void modifyTableReplicatedStorage(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildReplicatedStorage(); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyReplicatedStorage(info); - } - - // The caller need to hold the db write lock - public void modifyTableAutomaticBucketSize(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildBucketSize(); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyBucketSize(info); - } - - public void modifyTableMutableBucketNum(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildMutableBucketNum(); - - ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyMutableBucketNum(info); - } - - public void modifyTableEnableLoadProfile(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildEnableLoadProfile(); - - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyEnableLoadProfile(info); - } - - public void modifyTablePrimaryIndexCacheExpireSec(Database db, OlapTable table, Map properties) { - Locker locker = new Locker(); - Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); - TableProperty tableProperty = table.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - table.setTableProperty(tableProperty); - } else { - tableProperty.modifyTableProperties(properties); - } - tableProperty.buildPrimaryIndexCacheExpireSec(); - - ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), - properties); - GlobalStateMgr.getCurrentState().getEditLog().logModifyPrimaryIndexCacheExpireSec(info); - } - - public void modifyTableMeta(Database db, OlapTable table, Map properties, - TTabletMetaType metaType) { - if (metaType == TTabletMetaType.INMEMORY) { - modifyTableInMemoryMeta(db, table, properties); - } else if (metaType == TTabletMetaType.ENABLE_PERSISTENT_INDEX) { - modifyTableEnablePersistentIndexMeta(db, table, properties); - } else if (metaType == TTabletMetaType.WRITE_QUORUM) { - modifyTableWriteQuorum(db, table, properties); - } else if (metaType == TTabletMetaType.REPLICATED_STORAGE) { - modifyTableReplicatedStorage(db, table, properties); - } else if (metaType == TTabletMetaType.BUCKET_SIZE) { - modifyTableAutomaticBucketSize(db, table, properties); - } else if (metaType == TTabletMetaType.MUTABLE_BUCKET_NUM) { - modifyTableMutableBucketNum(db, table, properties); - } else if (metaType == TTabletMetaType.PRIMARY_INDEX_CACHE_EXPIRE_SEC) { - modifyTablePrimaryIndexCacheExpireSec(db, table, properties); - } else if (metaType == TTabletMetaType.ENABLE_LOAD_PROFILE) { - modifyTableEnableLoadProfile(db, table, properties); - } - } - - public void setHasForbiddenGlobalDict(String dbName, String tableName, boolean isForbit) throws DdlException { - Map property = new HashMap<>(); - Database db = getDb(dbName); - if (db == null) { - throw new DdlException("the DB " + dbName + " is not exist"); - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - Table table = getTable(db.getFullName(), tableName); - if (table == null) { - throw new DdlException("the DB " + dbName + " table: " + tableName + "isn't exist"); - } - - if (table instanceof OlapTable) { - OlapTable olapTable = (OlapTable) table; - olapTable.setHasForbiddenGlobalDict(isForbit); - if (isForbit) { - property.put(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE, PropertyAnalyzer.DISABLE_LOW_CARD_DICT); - IDictManager.getInstance().disableGlobalDict(olapTable.getId()); - } else { - property.put(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE, PropertyAnalyzer.ABLE_LOW_CARD_DICT); - IDictManager.getInstance().enableGlobalDict(olapTable.getId()); - } - ModifyTablePropertyOperationLog info = - new ModifyTablePropertyOperationLog(db.getId(), table.getId(), property); - GlobalStateMgr.getCurrentState().getEditLog().logSetHasForbiddenGlobalDict(info); - } - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - } - - public void replayModifyHiveTableColumn(short opCode, ModifyTableColumnOperationLog info) { - if (info.getDbName() == null) { - return; - } - String hiveExternalDb = info.getDbName(); - String hiveExternalTable = info.getTableName(); - LOG.info("replayModifyTableColumn hiveDb:{},hiveTable:{}", hiveExternalDb, hiveExternalTable); - List columns = info.getColumns(); - Database db = getDb(hiveExternalDb); - HiveTable table; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - Table tbl = getTable(db.getFullName(), hiveExternalTable); - table = (HiveTable) tbl; - table.setNewFullSchema(columns); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperationLog info) { - long dbId = info.getDbId(); - long tableId = info.getTableId(); - Map properties = info.getProperties(); - String comment = info.getComment(); - - Database db = getDb(dbId); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), tableId); - if (opCode == OperationType.OP_SET_FORBIDDEN_GLOBAL_DICT) { - String enAble = properties.get(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE); - Preconditions.checkState(enAble != null); - if (olapTable != null) { - if (enAble.equals(PropertyAnalyzer.DISABLE_LOW_CARD_DICT)) { - olapTable.setHasForbiddenGlobalDict(true); - IDictManager.getInstance().disableGlobalDict(olapTable.getId()); - } else { - olapTable.setHasForbiddenGlobalDict(false); - IDictManager.getInstance().enableGlobalDict(olapTable.getId()); - } - } - } else { - TableProperty tableProperty = olapTable.getTableProperty(); - if (tableProperty == null) { - tableProperty = new TableProperty(properties); - olapTable.setTableProperty(tableProperty.buildProperty(opCode)); - } else { - tableProperty.modifyTableProperties(properties); - tableProperty.buildProperty(opCode); - } - - if (StringUtils.isNotEmpty(comment)) { - olapTable.setComment(comment); - } - - // need to replay partition info meta - if (opCode == OperationType.OP_MODIFY_IN_MEMORY) { - for (Partition partition : olapTable.getPartitions()) { - olapTable.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory()); - } - } else if (opCode == OperationType.OP_MODIFY_REPLICATION_NUM) { - // update partition replication num if this table is unpartitioned table - PartitionInfo partitionInfo = olapTable.getPartitionInfo(); - if (partitionInfo.getType() == PartitionType.UNPARTITIONED) { - String partitionName = olapTable.getName(); - Partition partition = olapTable.getPartition(partitionName); - if (partition != null) { - partitionInfo.setReplicationNum(partition.getId(), tableProperty.getReplicationNum()); - } - } - } else if (opCode == OperationType.OP_MODIFY_ENABLE_PERSISTENT_INDEX) { - olapTable.setEnablePersistentIndex(tableProperty.enablePersistentIndex()); - if (olapTable.isCloudNativeTable()) { - olapTable.setPersistentIndexType(tableProperty.getPersistentIndexType()); - } - } else if (opCode == OperationType.OP_MODIFY_PRIMARY_INDEX_CACHE_EXPIRE_SEC) { - olapTable.setPrimaryIndexCacheExpireSec(tableProperty.primaryIndexCacheExpireSec()); - } else if (opCode == OperationType.OP_MODIFY_BINLOG_CONFIG) { - if (!olapTable.isBinlogEnabled()) { - olapTable.clearBinlogAvailableVersion(); - } - } - } - } catch (Exception ex) { - LOG.warn("The replay log failed and this log was ignored.", ex); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - @Override - public void createView(CreateViewStmt stmt) throws DdlException { - String dbName = stmt.getDbName(); - 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); - } - // check if table exists in db - boolean existed = false; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - if (getTable(db.getFullName(), tableName) != null) { - existed = true; - if (stmt.isSetIfNotExists()) { - LOG.info("create view[{}] which already exists", tableName); - return; - } else if (stmt.isReplace()) { - LOG.info("view {} already exists, need to replace it", tableName); - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName); - } - } - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - - if (existed) { - // already existed, need to alter the view - AlterViewStmt alterViewStmt = AlterViewStmt.fromReplaceStmt(stmt); - new AlterJobExecutor().process(alterViewStmt, ConnectContext.get()); - LOG.info("replace view {} successfully", tableName); - } else { - List columns = stmt.getColumns(); - long tableId = getNextId(); - View view = new View(tableId, tableName, columns); - view.setComment(stmt.getComment()); - view.setInlineViewDefWithSqlMode(stmt.getInlineViewDef(), - ConnectContext.get().getSessionVariable().getSqlMode()); - // init here in case the stmt string from view.toSql() has some syntax error. - try { - view.init(); - } catch (UserException e) { - throw new DdlException("failed to init view stmt", e); - } - - onCreate(db, view, "", stmt.isSetIfNotExists()); - LOG.info("successfully create view[" + tableName + "-" + view.getId() + "]"); - } - } - - public void replayUpdateClusterAndBackends(BackendIdsUpdateInfo info) { - for (long id : info.getBackendList()) { - final Backend backend = stateMgr.getNodeMgr().getClusterInfo().getBackend(id); - backend.setDecommissioned(false); - backend.setBackendState(Backend.BackendState.free); - } - } - - /* - * Truncate specified table or partitions. - * The main idea is: - * - * 1. using the same schema to create new table(partitions) - * 2. use the new created table(partitions) to replace the old ones. - * - * if no partition specified, it will truncate all partitions of this table, including all temp partitions, - * otherwise, it will only truncate those specified partitions. - * - */ - @Override - public void truncateTable(TruncateTableStmt truncateTableStmt, ConnectContext context) throws DdlException { - TableRef tblRef = truncateTableStmt.getTblRef(); - TableName dbTbl = tblRef.getName(); - // check, and save some info which need to be checked again later - Map origPartitions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - OlapTable copiedTbl; - Database db = getDb(dbTbl.getDb()); - if (db == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_DB_ERROR, dbTbl.getDb()); - } - - boolean truncateEntireTable = tblRef.getPartitionNames() == null; - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); - try { - Table table = MetaUtils.getSessionAwareTable(context, db, dbTbl); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, dbTbl.getTbl()); - } - - if (!table.isOlapOrCloudNativeTable()) { - throw new DdlException("Only support truncate OLAP table or LAKE table"); - } - - OlapTable olapTable = (OlapTable) table; - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); - } - - 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); - GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); - } - } else { - for (Partition partition : olapTable.getPartitions()) { - origPartitions.put(partition.getName(), partition); - GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(partition.getId()); - } - } - - copiedTbl = getShadowCopyTable(olapTable); - } finally { - locker.unLockDatabase(db.getId(), LockType.READ); - } - - // 2. use the copied table to create partitions - List newPartitions = Lists.newArrayListWithCapacity(origPartitions.size()); - // tabletIdSet to save all newly created tablet ids. - Set tabletIdSet = Sets.newHashSet(); - try { - for (Map.Entry entry : origPartitions.entrySet()) { - long oldPartitionId = entry.getValue().getId(); - long newPartitionId = getNextId(); - String newPartitionName = entry.getKey(); - - PartitionInfo partitionInfo = copiedTbl.getPartitionInfo(); - partitionInfo.setTabletType(newPartitionId, partitionInfo.getTabletType(oldPartitionId)); - partitionInfo.setIsInMemory(newPartitionId, partitionInfo.getIsInMemory(oldPartitionId)); - partitionInfo.setReplicationNum(newPartitionId, partitionInfo.getReplicationNum(oldPartitionId)); - partitionInfo.setDataProperty(newPartitionId, partitionInfo.getDataProperty(oldPartitionId)); - - if (copiedTbl.isCloudNativeTable()) { - partitionInfo.setDataCacheInfo(newPartitionId, - partitionInfo.getDataCacheInfo(oldPartitionId)); - } - - copiedTbl.setDefaultDistributionInfo(entry.getValue().getDistributionInfo()); - - Partition newPartition = - createPartition(db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, - ConnectContext.get().getCurrentWarehouseId()); - newPartitions.add(newPartition); - } - buildPartitions(db, copiedTbl, newPartitions.stream().map(Partition::getSubPartitions) - .flatMap(p -> p.stream()).collect(Collectors.toList()), ConnectContext.get().getCurrentWarehouseId()); - } catch (DdlException e) { - deleteUselessTablets(tabletIdSet); - throw e; - } - Preconditions.checkState(origPartitions.size() == newPartitions.size()); - - // all partitions are created successfully, try to replace the old partitions. - // before replacing, we need to check again. - // Things may be changed outside the database lock. - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), copiedTbl.getId()); - if (olapTable == null) { - throw new DdlException("Table[" + copiedTbl.getName() + "] is dropped"); - } - - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - throw InvalidOlapTableStateException.of(olapTable.getState(), olapTable.getName()); - } - - // check partitions - for (Map.Entry entry : origPartitions.entrySet()) { - Partition partition = olapTable.getPartition(entry.getValue().getId()); - if (partition == null || !partition.getName().equalsIgnoreCase(entry.getKey())) { - throw new DdlException("Partition [" + entry.getKey() + "] is changed during truncating table, " + - "please retry"); - } - } - - // check if meta changed - // rollup index may be added or dropped, and schema may be changed during creating partition operation. - boolean metaChanged = false; - if (olapTable.getIndexNameToId().size() != copiedTbl.getIndexNameToId().size()) { - metaChanged = true; - } else { - // compare schemaHash - Map copiedIndexIdToSchemaHash = copiedTbl.getIndexIdToSchemaHash(); - for (Map.Entry entry : olapTable.getIndexIdToSchemaHash().entrySet()) { - long indexId = entry.getKey(); - if (!copiedIndexIdToSchemaHash.containsKey(indexId)) { - metaChanged = true; - break; - } - if (!copiedIndexIdToSchemaHash.get(indexId).equals(entry.getValue())) { - metaChanged = true; - break; - } - } - } - - if (olapTable.getDefaultDistributionInfo().getType() != copiedTbl.getDefaultDistributionInfo().getType()) { - metaChanged = true; - } - - if (metaChanged) { - throw new DdlException("Table[" + copiedTbl.getName() + "]'s meta has been changed. try again."); - } - - // replace - truncateTableInternal(olapTable, newPartitions, truncateEntireTable, false); - - try { - colocateTableIndex.updateLakeTableColocationInfo(olapTable, true /* isJoin */, - null /* expectGroupId */); - } catch (DdlException e) { - LOG.info("table {} update colocation info failed when truncate table, {}", olapTable.getId(), e.getMessage()); - } - - // write edit log - TruncateTableInfo info = new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions, - truncateEntireTable); - GlobalStateMgr.getCurrentState().getEditLog().logTruncateTable(info); - - // refresh mv - Set relatedMvs = olapTable.getRelatedMaterializedViews(); - for (MvId mvId : relatedMvs) { - MaterializedView materializedView = (MaterializedView) getTable(db.getId(), mvId.getId()); - if (materializedView == null) { - LOG.warn("Table related materialized view {} can not be found", mvId.getId()); - continue; - } - if (materializedView.isLoadTriggeredRefresh()) { - refreshMaterializedView(db.getFullName(), getTable(db.getId(), mvId.getId()).getName(), false, null, - Constants.TaskRunPriority.NORMAL.value(), true, false); - } - } - } catch (DdlException e) { - deleteUselessTablets(tabletIdSet); - throw e; - } catch (MetaNotFoundException e) { - LOG.warn("Table related materialized view can not be found", e); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - - LOG.info("finished to truncate table {}, partitions: {}", - tblRef.getName().toSql(), tblRef.getPartitionNames()); - } - - private void deleteUselessTablets(Set tabletIdSet) { - // create partition failed, remove all newly created tablets. - // For lakeTable, shards cleanup is taken care in ShardDeleter. - for (Long tabletId : tabletIdSet) { - GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); - } - } - - private void truncateTableInternal(OlapTable olapTable, List newPartitions, - boolean isEntireTable, boolean isReplay) { - // use new partitions to replace the old ones. - Set oldTablets = Sets.newHashSet(); - for (Partition newPartition : newPartitions) { - Partition oldPartition = olapTable.replacePartition(newPartition); - for (PhysicalPartition physicalPartition : oldPartition.getSubPartitions()) { - // save old tablets to be removed - for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - // let HashSet do the deduplicate work - oldTablets.addAll(index.getTablets()); - } - } - } - - if (isEntireTable) { - // drop all temp partitions - olapTable.dropAllTempPartitions(); - } - - // remove the tablets in old partitions - for (Tablet tablet : oldTablets) { - TabletInvertedIndex index = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - index.deleteTablet(tablet.getId()); - // Ensure that only the leader records truncate information. - // TODO(yangzaorang): the information will be lost when failover occurs. The probability of this case - // happening is small, and the trash data will be deleted by BE anyway, but we need to find a better - // solution. - if (!isReplay) { - index.markTabletForceDelete(tablet); - } - } - } - - public void replayTruncateTable(TruncateTableInfo info) { - Database db = getDb(info.getDbId()); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable olapTable = (OlapTable) getTable(db.getId(), info.getTblId()); - truncateTableInternal(olapTable, info.getPartitions(), info.isEntireTable(), true); - - if (!GlobalStateMgr.isCheckpointThread()) { - // add tablet to inverted index - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (Partition partition : info.getPartitions()) { - long partitionId = partition.getId(); - TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty( - partitionId).getStorageMedium(); - for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { - for (MaterializedIndex mIndex : physicalPartition.getMaterializedIndices( - MaterializedIndex.IndexExtState.ALL)) { - long indexId = mIndex.getId(); - int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(), - physicalPartition.getId(), indexId, schemaHash, medium, - olapTable.isCloudNativeTableOrMaterializedView()); - for (Tablet tablet : mIndex.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.addTablet(tabletId, tabletMeta); - if (olapTable.isOlapTable()) { - for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { - invertedIndex.addReplica(tabletId, replica); - } - } - } - } - } - } - } - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } - } - - public void replayBackendTabletsInfo(BackendTabletsInfo backendTabletsInfo) { - List> tabletsWithSchemaHash = backendTabletsInfo.getTabletSchemaHash(); - if (!tabletsWithSchemaHash.isEmpty()) { - // In previous version, we save replica info in `tabletsWithSchemaHash`, - // but it is wrong because we can not get replica from `tabletInvertedIndex` when doing checkpoint, - // because when doing checkpoint, the tabletInvertedIndex is not initialized at all. - // - // So we can only discard this information, in this case, it is equivalent to losing the record of these operations. - // But it doesn't matter, these records are currently only used to record whether a replica is in a bad state. - // This state has little effect on the system, and it can be restored after the system has processed the bad state replica. - for (Pair tabletInfo : tabletsWithSchemaHash) { - LOG.warn("find an old backendTabletsInfo for tablet {}, ignore it", tabletInfo.first); - } - return; - } - - // in new version, replica info is saved here. - // 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; - } - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - OlapTable tbl = (OlapTable) getTable(db.getId(), info.getTableId()); - if (tbl == null) { - continue; - } - Partition partition = tbl.getPartition(info.getPartitionId()); - if (partition == null) { - continue; - } - MaterializedIndex mindex = partition.getIndex(info.getIndexId()); - if (mindex == null) { - continue; - } - LocalTablet tablet = (LocalTablet) mindex.getTablet(info.getTabletId()); - if (tablet == null) { - continue; - } - Replica replica = tablet.getReplicaById(info.getReplicaId()); - if (replica != null) { - replica.setBad(true); - LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying", - info.getReplicaId(), info.getTabletId(), info.getBackendId()); - } - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } + }); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); } } - // Convert table's distribution type from random to hash. - // random distribution is no longer supported. - public void convertDistributionType(Database db, OlapTable tbl) throws DdlException { - TableInfo tableInfo = TableInfo.createForModifyDistribution(db.getId(), tbl.getId()); - GlobalStateMgr.getCurrentState().getEditLog().logModifyDistributionType(tableInfo); - LOG.info("finished to modify distribution type of table: " + tbl.getName()); + @Override + public void renamePartition(TableInfo tableInfo) { + GlobalStateMgr.getCurrentState().getEditLog().logPartitionRename(tableInfo); } - public void replayConvertDistributionType(TableInfo tableInfo) { - Database db = getDb(tableInfo.getDbId()); + public void replayRenamePartition(TableInfo tableInfo) { + long dbId = tableInfo.getDbId(); + long tableId = tableInfo.getTableId(); + long partitionId = tableInfo.getPartitionId(); + String newPartitionName = tableInfo.getNewPartitionName(); + + Database db = getDb(dbId); Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.WRITE); try { - OlapTable tbl = (OlapTable) getTable(db.getId(), tableInfo.getTableId()); - LOG.info("replay modify distribution type of table: " + tbl.getName()); + OlapTable table = (OlapTable) getTable(db.getId(), tableId); + Partition partition = table.getPartition(partitionId); + table.renamePartition(partition.getName(), newPartitionName); + LOG.info("replay rename partition[{}] to {}", partition.getName(), newPartitionName); } finally { locker.unLockDatabase(db.getId(), LockType.WRITE); } } - /* - * The entry of replacing partitions with temp partitions. - */ - public void replaceTempPartition(Database db, String tableName, ReplacePartitionClause clause) throws DdlException { - List partitionNames = clause.getPartitionNames(); - // duplicate temp partition will cause Incomplete transaction - List tempPartitionNames = - clause.getTempPartitionNames().stream().distinct().collect(Collectors.toList()); - - boolean isStrictRange = clause.isStrictRange(); - boolean useTempPartitionName = clause.useTempPartitionName(); - Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.WRITE); - try { - Table table = getTable(db.getFullName(), tableName); - if (table == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName); - } - - if (!table.isOlapOrCloudNativeTable()) { - throw new DdlException("Table[" + tableName + "] is not OLAP table or LAKE table"); - } - - OlapTable olapTable = (OlapTable) table; - // check partition exist - for (String partName : partitionNames) { - if (!olapTable.checkPartitionNameExist(partName, false)) { - throw new DdlException("Partition[" + partName + "] does not exist"); - } - } - for (String partName : tempPartitionNames) { - if (!olapTable.checkPartitionNameExist(partName, true)) { - throw new DdlException("Temp partition[" + partName + "] does not exist"); - } - } - - partitionNames.stream().forEach(e -> - GlobalStateMgr.getCurrentState().getAnalyzeMgr().recordDropPartition(olapTable.getPartition(e).getId())); - olapTable.replaceTempPartitions(partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); - - // write log - ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(), - partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName); - GlobalStateMgr.getCurrentState().getEditLog().logReplaceTempPartition(info); - LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", - clause.getPartitionNames(), clause.getTempPartitionNames(), tableName); - } finally { - locker.unLockDatabase(db.getId(), LockType.WRITE); - } + @Override + public void replaceTempPartition(ReplacePartitionOperationLog info) { + GlobalStateMgr.getCurrentState().getEditLog().logReplaceTempPartition(info); } public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog) { @@ -4554,284 +894,209 @@ public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempP } } - // entry of checking tablets operation - public void checkTablets(AdminCheckTabletsStmt stmt) { - AdminCheckTabletsStmt.CheckType type = stmt.getType(); - if (type == AdminCheckTabletsStmt.CheckType.CONSISTENCY) { - stateMgr.getConsistencyChecker().addTabletsToCheck(stmt.getTabletIds()); - } - } - - // Set specified replica's status. If replica does not exist, just ignore it. - public void setReplicaStatus(AdminSetReplicaStatusStmt stmt) { - long tabletId = stmt.getTabletId(); - long backendId = stmt.getBackendId(); - Replica.ReplicaStatus status = stmt.getStatus(); - setReplicaStatusInternal(tabletId, backendId, status, false); - } - - public void replaySetReplicaStatus(SetReplicaStatusOperationLog log) { - setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true); - } - - private void setReplicaStatusInternal(long tabletId, long backendId, Replica.ReplicaStatus status, - boolean isReplay) { - TabletMeta meta = stateMgr.getTabletInvertedIndex().getTabletMeta(tabletId); - if (meta == null) { - LOG.info("tablet {} does not exist", tabletId); - return; - } - long dbId = meta.getDbId(); + public void replayRecoverPartition(RecoverInfo info) { + long dbId = info.getDbId(); Database db = getDb(dbId); - if (db == null) { - LOG.info("database {} of tablet {} does not exist", dbId, tabletId); - return; - } Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.WRITE); try { - Replica replica = stateMgr.getTabletInvertedIndex().getReplica(tabletId, backendId); - if (replica == null) { - LOG.info("replica of tablet {} does not exist", tabletId); - return; - } - if (status == Replica.ReplicaStatus.BAD || status == Replica.ReplicaStatus.OK) { - if (replica.setBadForce(status == Replica.ReplicaStatus.BAD)) { - if (!isReplay) { - // Put this tablet into urgent table so that it can be repaired ASAP. - stateMgr.getTabletChecker().setTabletForUrgentRepair(dbId, meta.getTableId(), - meta.getPartitionId()); - SetReplicaStatusOperationLog log = - new SetReplicaStatusOperationLog(backendId, tabletId, status); - GlobalStateMgr.getCurrentState().getEditLog().logSetReplicaStatus(log); - } - LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}", - replica.getId(), tabletId, backendId, status, isReplay); - } - } + Table table = getTable(db.getId(), info.getTableId()); + recycleBin.replayRecoverPartition((OlapTable) table, info.getPartitionId()); } finally { locker.unLockDatabase(db.getId(), LockType.WRITE); } } - public void setPartitionVersion(AdminSetPartitionVersionStmt stmt) { - Database database = getDb(stmt.getTableName().getDb()); - if (database == null) { - throw ErrorReportException.report(ErrorCode.ERR_BAD_DB_ERROR, stmt.getTableName().getDb()); + @Override + public void setPartitionVersion(PartitionVersionRecoveryInfo info) { + GlobalStateMgr.getCurrentState().getEditLog().logRecoverPartitionVersion(info); + } + + @Override + public void addSubPartitionLog(Database db, OlapTable olapTable, Partition partition, + List subPartitions) { + List partitionInfoV2List = Lists.newArrayList(); + for (PhysicalPartition subPartition : subPartitions) { + PhysicalPartitionPersistInfoV2 info = new PhysicalPartitionPersistInfoV2( + db.getId(), olapTable.getId(), partition.getId(), subPartition); + partitionInfoV2List.add(info); } - Locker locker = new Locker(); - locker.lockDatabase(database.getId(), LockType.WRITE); - try { - Table table = getTable(database.getFullName(), stmt.getTableName().getTbl()); - if (table == null) { - throw ErrorReportException.report(ErrorCode.ERR_BAD_TABLE_ERROR, stmt.getTableName().getTbl()); - } - if (!table.isOlapTableOrMaterializedView()) { - throw ErrorReportException.report(ErrorCode.ERR_NOT_OLAP_TABLE, stmt.getTableName().getTbl()); - } - PhysicalPartition physicalPartition; - OlapTable olapTable = (OlapTable) table; - if (stmt.getPartitionId() != -1) { - physicalPartition = olapTable.getPhysicalPartition(stmt.getPartitionId()); - if (physicalPartition == null) { - throw ErrorReportException.report(ErrorCode.ERR_NO_SUCH_PARTITION, stmt.getPartitionName()); - } - } else { - Partition partition = olapTable.getPartition(stmt.getPartitionName()); - if (partition == null) { - throw ErrorReportException.report(ErrorCode.ERR_NO_SUCH_PARTITION, stmt.getPartitionName()); - } - if (partition.getSubPartitions().size() >= 2) { - throw ErrorReportException.report(ErrorCode.ERR_MULTI_SUB_PARTITION, stmt.getPartitionName()); - } - physicalPartition = partition; - } + AddSubPartitionsInfoV2 infos = new AddSubPartitionsInfoV2(partitionInfoV2List); + GlobalStateMgr.getCurrentState().getEditLog().logAddSubPartitions(infos); - long visibleVersionTime = System.currentTimeMillis(); - physicalPartition.setVisibleVersion(stmt.getVersion(), visibleVersionTime); - physicalPartition.setNextVersion(stmt.getVersion() + 1); + for (PhysicalPartition subPartition : subPartitions) { + LOG.info("succeed in creating sub partitions[{}]", subPartition); + } + } - PartitionVersion partitionVersion = new PartitionVersion(database.getId(), table.getId(), - physicalPartition.getId(), stmt.getVersion()); - for (MaterializedIndex index : physicalPartition.getMaterializedIndices(IndexExtState.VISIBLE)) { - for (Tablet tablet : index.getTablets()) { - if (!(tablet instanceof LocalTablet)) { - continue; - } + public void replayAddSubPartition(PhysicalPartitionPersistInfoV2 info) throws DdlException { + Database db = this.getDb(info.getDbId()); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) getTable(db.getId(), info.getTableId()); + Partition partition = olapTable.getPartition(info.getPartitionId()); + PhysicalPartition physicalPartition = info.getPhysicalPartition(); + partition.addSubPartition(physicalPartition); + olapTable.addPhysicalPartition(physicalPartition); - LocalTablet localTablet = (LocalTablet) tablet; - for (Replica replica : localTablet.getAllReplicas()) { - if (replica.getVersion() > stmt.getVersion() && localTablet.getAllReplicas().size() > 1) { - replica.setBad(true); - LOG.warn("set tablet: {} on backend: {} to bad, " + - "because its version: {} is higher than partition visible version: {}", - tablet.getId(), replica.getBackendId(), replica.getVersion(), stmt.getVersion()); + if (!isCheckpointThread()) { + // add to inverted index + TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); + for (MaterializedIndex index : physicalPartition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { + long indexId = index.getId(); + int schemaHash = olapTable.getSchemaHashByIndexId(indexId); + TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), info.getPartitionId(), + physicalPartition.getId(), index.getId(), schemaHash, olapTable.getPartitionInfo().getDataProperty( + info.getPartitionId()).getStorageMedium(), false); + for (Tablet tablet : index.getTablets()) { + long tabletId = tablet.getId(); + invertedIndex.addTablet(tabletId, tabletMeta); + // modify some logic + if (tablet instanceof LocalTablet) { + for (Replica replica : ((LocalTablet) tablet).getImmutableReplicas()) { + invertedIndex.addReplica(tabletId, replica); + } } } } } - GlobalStateMgr.getCurrentState().getEditLog().logRecoverPartitionVersion( - new PartitionVersionRecoveryInfo(Lists.newArrayList(partitionVersion), visibleVersionTime)); - LOG.info("Successfully set partition: {} version to {}, table: {}, db: {}", - stmt.getPartitionName(), stmt.getVersion(), table.getName(), database.getFullName()); } finally { - locker.unLockDatabase(database.getId(), LockType.WRITE); + locker.unLockDatabase(db.getId(), LockType.WRITE); } } - public void onEraseDatabase(long dbId) { - // remove database transaction manager - stateMgr.getGlobalTransactionMgr().removeDatabaseTransactionMgr(dbId); - // unbind db to storage volume - stateMgr.getStorageVolumeMgr().unbindDbToStorageVolume(dbId); - } + public void replayRenameRollup(TableInfo tableInfo) { + long dbId = tableInfo.getDbId(); + long tableId = tableInfo.getTableId(); + long indexId = tableInfo.getIndexId(); + String newRollupName = tableInfo.getNewRollupName(); - public void onErasePartition(Partition partition) { - // remove tablet in inverted index - TabletInvertedIndex invertedIndex = GlobalStateMgr.getCurrentState().getTabletInvertedIndex(); - for (MaterializedIndex index : partition.getMaterializedIndices(MaterializedIndex.IndexExtState.ALL)) { - for (Tablet tablet : index.getTablets()) { - long tabletId = tablet.getId(); - invertedIndex.deleteTablet(tabletId); - } - } - } + Database db = getDb(dbId); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable table = (OlapTable) getTable(db.getId(), tableId); + String rollupName = table.getIndexNameById(indexId); + Map indexNameToIdMap = table.getIndexNameToId(); + indexNameToIdMap.remove(rollupName); + indexNameToIdMap.put(newRollupName, indexId); - // for test only - @VisibleForTesting - public void clear() { - if (idToDb != null) { - idToDb.clear(); - } - if (fullNameToDb != null) { - fullNameToDb.clear(); + LOG.info("replay rename rollup[{}] to {}", rollupName, newRollupName); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); } - System.gc(); } - @VisibleForTesting - public OlapTable getCopiedTable(Database db, OlapTable olapTable, List sourcePartitionIds, - Map origPartitions, boolean isOptimize) { - OlapTable copiedTbl; + public void replayRenameColumn(ColumnRenameInfo columnRenameInfo) throws DdlException { + long dbId = columnRenameInfo.getDbId(); + long tableId = columnRenameInfo.getTableId(); + String colName = columnRenameInfo.getColumnName(); + String newColName = columnRenameInfo.getNewColumnName(); + Database db = getDb(dbId); Locker locker = new Locker(); - locker.lockDatabase(db.getId(), LockType.READ); + locker.lockDatabase(db.getId(), LockType.WRITE); try { - if (olapTable.getState() != OlapTable.OlapTableState.NORMAL) { - if (!isOptimize || olapTable.getState() != OlapTable.OlapTableState.SCHEMA_CHANGE) { - throw new RuntimeException("Table' state is not NORMAL: " + olapTable.getState() - + ", tableId:" + olapTable.getId() + ", tabletName:" + olapTable.getName()); - } - } - for (Long id : sourcePartitionIds) { - origPartitions.put(id, olapTable.getPartition(id).getName()); - } - copiedTbl = getShadowCopyTable(olapTable); + OlapTable olapTable = (OlapTable) getTable(db.getId(), tableId); + olapTable.renameColumn(colName, newColName); + LOG.info("replay rename column[{}] to {}", colName, newColName); } finally { - locker.unLockDatabase(db.getId(), LockType.READ); + locker.unLockDatabase(db.getId(), LockType.WRITE); } - return copiedTbl; - } - - @VisibleForTesting - public OlapTable getCopiedTable(Database db, OlapTable olapTable, List sourcePartitionIds, - Map origPartitions) { - return getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions, false); } - private OlapTable getShadowCopyTable(OlapTable olapTable) { - OlapTable copiedTable; - if (olapTable instanceof LakeMaterializedView) { - copiedTable = new LakeMaterializedView(); - } else if (olapTable instanceof MaterializedView) { - copiedTable = new MaterializedView(); - } else if (olapTable instanceof LakeTable) { - copiedTable = new LakeTable(); - } else { - copiedTable = new OlapTable(); + public void replayModifyHiveTableColumn(short opCode, ModifyTableColumnOperationLog info) { + if (info.getDbName() == null) { + return; + } + String hiveExternalDb = info.getDbName(); + String hiveExternalTable = info.getTableName(); + LOG.info("replayModifyTableColumn hiveDb:{},hiveTable:{}", hiveExternalDb, hiveExternalTable); + List columns = info.getColumns(); + Database db = getDb(hiveExternalDb); + HiveTable table; + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + Table tbl = getTable(db.getFullName(), hiveExternalTable); + table = (HiveTable) tbl; + table.setNewFullSchema(columns); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); } - - olapTable.copyOnlyForQuery(copiedTable); - return copiedTable; } - @VisibleForTesting - public List getNewPartitionsFromPartitions(Database db, OlapTable olapTable, - List sourcePartitionIds, - Map origPartitions, OlapTable copiedTbl, - String namePostfix, Set tabletIdSet, - List tmpPartitionIds, DistributionDesc distributionDesc, - long warehouseId) - throws DdlException { - List newPartitions = Lists.newArrayListWithCapacity(sourcePartitionIds.size()); - for (int i = 0; i < sourcePartitionIds.size(); ++i) { - long newPartitionId = tmpPartitionIds.get(i); - long sourcePartitionId = sourcePartitionIds.get(i); - String newPartitionName = origPartitions.get(sourcePartitionId) + namePostfix; - if (olapTable.checkPartitionNameExist(newPartitionName, true)) { - // to prevent creating the same partitions when failover - // this will happen when OverwriteJob crashed after created temp partitions, - // but before changing to PREPARED state - LOG.warn("partition:{} already exists in table:{}", newPartitionName, olapTable.getName()); - continue; - } - PartitionInfo partitionInfo = copiedTbl.getPartitionInfo(); - partitionInfo.setTabletType(newPartitionId, partitionInfo.getTabletType(sourcePartitionId)); - partitionInfo.setIsInMemory(newPartitionId, partitionInfo.getIsInMemory(sourcePartitionId)); - partitionInfo.setReplicationNum(newPartitionId, partitionInfo.getReplicationNum(sourcePartitionId)); - partitionInfo.setDataProperty(newPartitionId, partitionInfo.getDataProperty(sourcePartitionId)); - if (copiedTbl.isCloudNativeTableOrMaterializedView()) { - partitionInfo.setDataCacheInfo(newPartitionId, partitionInfo.getDataCacheInfo(sourcePartitionId)); - } + public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperationLog info) { + long dbId = info.getDbId(); + long tableId = info.getTableId(); + Map properties = info.getProperties(); + String comment = info.getComment(); - Partition newPartition = null; - if (distributionDesc != null) { - DistributionInfo distributionInfo = distributionDesc.toDistributionInfo(olapTable.getColumns()); - if (distributionInfo.getBucketNum() == 0) { - Partition sourcePartition = olapTable.getPartition(sourcePartitionId); - olapTable.optimizeDistribution(distributionInfo, sourcePartition); + Database db = getDb(dbId); + Locker locker = new Locker(); + locker.lockDatabase(db.getId(), LockType.WRITE); + try { + OlapTable olapTable = (OlapTable) getTable(db.getId(), tableId); + if (opCode == OperationType.OP_SET_FORBIDDEN_GLOBAL_DICT) { + String enAble = properties.get(PropertyAnalyzer.ENABLE_LOW_CARD_DICT_TYPE); + Preconditions.checkState(enAble != null); + if (olapTable != null) { + if (enAble.equals(PropertyAnalyzer.DISABLE_LOW_CARD_DICT)) { + olapTable.setHasForbiddenGlobalDict(true); + IDictManager.getInstance().disableGlobalDict(olapTable.getId()); + } else { + olapTable.setHasForbiddenGlobalDict(false); + IDictManager.getInstance().enableGlobalDict(olapTable.getId()); + } } - newPartition = createPartition( - db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, distributionInfo, warehouseId); } else { - newPartition = createPartition(db, copiedTbl, newPartitionId, newPartitionName, null, tabletIdSet, warehouseId); - } + TableProperty tableProperty = olapTable.getTableProperty(); + if (tableProperty == null) { + tableProperty = new TableProperty(properties); + olapTable.setTableProperty(tableProperty.buildProperty(opCode)); + } else { + tableProperty.modifyTableProperties(properties); + tableProperty.buildProperty(opCode); + } - newPartitions.add(newPartition); - } - return newPartitions; - } + if (StringUtils.isNotEmpty(comment)) { + olapTable.setComment(comment); + } - // create new partitions from source partitions. - // new partitions have the same indexes as source partitions. - public List createTempPartitionsFromPartitions(Database db, Table table, - String namePostfix, List sourcePartitionIds, - List tmpPartitionIds, DistributionDesc distributionDesc, - long warehouseId) { - Preconditions.checkState(table instanceof OlapTable); - OlapTable olapTable = (OlapTable) table; - Map origPartitions = Maps.newHashMap(); - OlapTable copiedTbl = getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions, distributionDesc != null); - copiedTbl.setDefaultDistributionInfo(olapTable.getDefaultDistributionInfo()); - - // 2. use the copied table to create partitions - List newPartitions = null; - // tabletIdSet to save all newly created tablet ids. - Set tabletIdSet = Sets.newHashSet(); - try { - newPartitions = getNewPartitionsFromPartitions(db, olapTable, sourcePartitionIds, origPartitions, - copiedTbl, namePostfix, tabletIdSet, tmpPartitionIds, distributionDesc, warehouseId); - buildPartitions(db, copiedTbl, newPartitions.stream().map(Partition::getSubPartitions) - .flatMap(p -> p.stream()).collect(Collectors.toList()), warehouseId); - } catch (Exception e) { - // create partition failed, remove all newly created tablets - for (Long tabletId : tabletIdSet) { - GlobalStateMgr.getCurrentState().getTabletInvertedIndex().deleteTablet(tabletId); + // need to replay partition info meta + if (opCode == OperationType.OP_MODIFY_IN_MEMORY) { + for (Partition partition : olapTable.getPartitions()) { + olapTable.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory()); + } + } else if (opCode == OperationType.OP_MODIFY_REPLICATION_NUM) { + // update partition replication num if this table is unpartitioned table + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + if (partitionInfo.getType() == PartitionType.UNPARTITIONED) { + String partitionName = olapTable.getName(); + Partition partition = olapTable.getPartition(partitionName); + if (partition != null) { + partitionInfo.setReplicationNum(partition.getId(), tableProperty.getReplicationNum()); + } + } + } else if (opCode == OperationType.OP_MODIFY_ENABLE_PERSISTENT_INDEX) { + olapTable.setEnablePersistentIndex(tableProperty.enablePersistentIndex()); + if (olapTable.isCloudNativeTable()) { + olapTable.setPersistentIndexType(tableProperty.getPersistentIndexType()); + } + } else if (opCode == OperationType.OP_MODIFY_PRIMARY_INDEX_CACHE_EXPIRE_SEC) { + olapTable.setPrimaryIndexCacheExpireSec(tableProperty.primaryIndexCacheExpireSec()); + } else if (opCode == OperationType.OP_MODIFY_BINLOG_CONFIG) { + if (!olapTable.isBinlogEnabled()) { + olapTable.clearBinlogAvailableVersion(); + } + } } - LOG.warn("create partitions from partitions failed.", e); - throw new RuntimeException("create partitions failed: " + e.getMessage(), e); + } catch (Exception ex) { + LOG.warn("The replay log failed and this log was ignored.", ex); + } finally { + locker.unLockDatabase(db.getId(), LockType.WRITE); } - return newPartitions; } public void replayDeleteAutoIncrementId(AutoIncrementInfo info) throws IOException { @@ -4967,7 +1232,7 @@ public void load(SRMetaBlockReader reader) throws IOException, SRMetaBlockExcept tableIdToIncrementId.put(tableId, id); } - recreateTabletInvertIndex(); + GlobalStateMgr.getCurrentState().getTabletInvertedIndex().recreateTabletInvertIndex(); GlobalStateMgr.getCurrentState().getEsRepository().loadTableFromCatalog(); } @@ -4976,6 +1241,18 @@ public void handleMVRepair(Database db, Table table, List, Long>> getSamples() { long totalCount = idToDb.values() diff --git a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastoreReplayer.java b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastoreReplayer.java new file mode 100644 index 0000000000000..8c030f2f5d0fd --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastoreReplayer.java @@ -0,0 +1,23 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +package com.starrocks.server; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class LocalMetastoreReplayer { + private static final Logger LOG = LogManager.getLogger(LocalMetastoreReplayer.class); + + +} diff --git a/fe/fe-core/src/main/java/com/starrocks/server/MetadataMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/MetadataMgr.java index e10ada5c431d3..bfb1eb915804c 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/MetadataMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/MetadataMgr.java @@ -62,6 +62,7 @@ import com.starrocks.connector.metadata.MetadataTable; import com.starrocks.connector.metadata.MetadataTableType; import com.starrocks.connector.statistics.ConnectorTableColumnStats; +import com.starrocks.meta.StarRocksMeta; import com.starrocks.qe.ConnectContext; import com.starrocks.sql.ast.AlterTableStmt; import com.starrocks.sql.ast.CleanTemporaryTableStmt; @@ -118,7 +119,7 @@ public synchronized ConnectorMetadata getConnectorMetadata(String catalogName, S } } - private final LocalMetastore localMetastore; + private final StarRocksMeta starRocksMeta; private final TemporaryTableMgr temporaryTableMgr; private final ConnectorMgr connectorMgr; private final ConnectorTblMetaInfoMgr connectorTblMetaInfoMgr; @@ -148,11 +149,11 @@ public QueryMetadatas load(String key) throws Exception { } }); - public MetadataMgr(LocalMetastore localMetastore, TemporaryTableMgr temporaryTableMgr, ConnectorMgr connectorMgr, + public MetadataMgr(StarRocksMeta starrocksMeta, TemporaryTableMgr temporaryTableMgr, ConnectorMgr connectorMgr, ConnectorTblMetaInfoMgr connectorTblMetaInfoMgr) { - Preconditions.checkNotNull(localMetastore, "localMetastore is null"); + Preconditions.checkNotNull(starrocksMeta, "localMetastore is null"); Preconditions.checkNotNull(temporaryTableMgr, "temporaryTableMgr is null"); - this.localMetastore = localMetastore; + this.starRocksMeta = starrocksMeta; this.temporaryTableMgr = temporaryTableMgr; this.connectorMgr = connectorMgr; this.connectorTblMetaInfoMgr = connectorTblMetaInfoMgr; @@ -180,7 +181,7 @@ public Optional getOptionalMetadata(String catalogName) { */ public Optional getOptionalMetadata(Optional queryId, String catalogName) { if (Strings.isNullOrEmpty(catalogName) || CatalogMgr.isInternalCatalog(catalogName)) { - return Optional.of(localMetastore); + return Optional.of(starRocksMeta); } CatalogConnector connector = connectorMgr.getConnector(catalogName); @@ -253,7 +254,7 @@ public Database getDb(String catalogName, String dbName) { } public Database getDb(Long databaseId) { - return localMetastore.getDb(databaseId); + return starRocksMeta.getDb(databaseId); } public List listTableNames(String catalogName, String dbName) { @@ -475,7 +476,7 @@ public void cleanTemporaryTables(UUID sessionId) { com.google.common.collect.Table allTables = temporaryTableMgr.getTemporaryTables(sessionId); for (Long databaseId : allTables.rowKeySet()) { - Database database = localMetastore.getDb(databaseId); + Database database = starRocksMeta.getDb(databaseId); if (database == null) { // database maybe dropped by force, we should clean temporary tables on it. temporaryTableMgr.dropTemporaryTables(sessionId, databaseId); @@ -539,7 +540,7 @@ public Optional getDatabase(BaseTableInfo baseTableInfo) { public Optional
getTable(BaseTableInfo baseTableInfo) { if (baseTableInfo.isInternalCatalog()) { - return Optional.ofNullable(localMetastore.getTable(baseTableInfo.getDbId(), baseTableInfo.getTableId())); + return Optional.ofNullable(starRocksMeta.getTable(baseTableInfo.getDbId(), baseTableInfo.getTableId())); } else { return Optional.ofNullable( getTable(baseTableInfo.getCatalogName(), baseTableInfo.getDbName(), baseTableInfo.getTableName())); @@ -576,11 +577,11 @@ public Table getTemporaryTable(UUID sessionId, String catalogName, Long database if (tableId == null) { return null; } - Database database = localMetastore.getDb(databaseId); + Database database = starRocksMeta.getDb(databaseId); if (database == null) { return null; } - return localMetastore.getTable(database.getId(), tableId); + return starRocksMeta.getTable(database.getId(), tableId); } public boolean tableExists(String catalogName, String dbName, String tblName) { diff --git a/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java b/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java index 65376890e29cb..a4b07c031cdf0 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java @@ -94,8 +94,7 @@ private OlapTableFactory() { @Override @NotNull public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt stmt) throws DdlException { - GlobalStateMgr stateMgr = metastore.getStateMgr(); - ColocateTableIndex colocateTableIndex = metastore.getColocateTableIndex(); + ColocateTableIndex colocateTableIndex = GlobalStateMgr.getCurrentState().getColocateTableIndex(); String tableName = stmt.getTableName(); if (stmt instanceof CreateTemporaryTableStmt) { @@ -107,7 +106,7 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt // create columns List baseSchema = stmt.getColumns(); - metastore.validateColumns(baseSchema); + GlobalStateMgr.getCurrentState().getStarRocksMeta().validateColumns(baseSchema); // create partition info PartitionDesc partitionDesc = stmt.getPartitionDesc(); @@ -118,18 +117,18 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt if (partitionDesc instanceof RangePartitionDesc) { RangePartitionDesc rangePartitionDesc = (RangePartitionDesc) partitionDesc; for (SingleRangePartitionDesc desc : rangePartitionDesc.getSingleRangePartitionDescs()) { - long partitionId = metastore.getNextId(); + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); partitionNameToId.put(desc.getPartitionName(), partitionId); } } else if (partitionDesc instanceof ListPartitionDesc) { ListPartitionDesc listPartitionDesc = (ListPartitionDesc) partitionDesc; - listPartitionDesc.findAllPartitionNames() - .forEach(partitionName -> partitionNameToId.put(partitionName, metastore.getNextId())); + listPartitionDesc.findAllPartitionNames().forEach( + partitionName -> partitionNameToId.put(partitionName, GlobalStateMgr.getCurrentState().getNextId())); } else if (partitionDesc instanceof ExpressionPartitionDesc) { ExpressionPartitionDesc expressionPartitionDesc = (ExpressionPartitionDesc) partitionDesc; for (SingleRangePartitionDesc desc : expressionPartitionDesc.getRangePartitionDesc() .getSingleRangePartitionDescs()) { - long partitionId = metastore.getNextId(); + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); partitionNameToId.put(desc.getPartitionName(), partitionId); } @@ -143,7 +142,7 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt // Automatic partitioning needs to ensure that at least one tablet is opened. if (partitionInfo.isAutomaticPartition()) { - long partitionId = metastore.getNextId(); + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); String replicateNum = String.valueOf(RunMode.defaultReplicationNum()); if (stmt.getProperties() != null) { replicateNum = stmt.getProperties().getOrDefault("replication_num", @@ -157,7 +156,7 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(stmt.getProperties())) { throw new DdlException("Only support dynamic partition properties on range partition table"); } - long partitionId = metastore.getNextId(); + long partitionId = GlobalStateMgr.getCurrentState().getNextId(); // use table name as single partition name partitionNameToId.put(tableName, partitionId); partitionInfo = new SinglePartitionInfo(); @@ -232,7 +231,8 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt throw new DdlException(String.format("Storage volume %s not exists", volume)); } String storageVolumeId = svm.getStorageVolumeIdOfTable(tableId); - metastore.setLakeStorageInfo(db, table, storageVolumeId, properties); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .setLakeStorageInfo(db, table, storageVolumeId, properties); } else { table = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, distributionInfo, indexes); } @@ -251,7 +251,7 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt table.setComment(stmt.getComment()); // set base index id - long baseIndexId = metastore.getNextId(); + long baseIndexId = GlobalStateMgr.getCurrentState().getNextId(); table.setBaseIndexId(baseIndexId); // get use light schema change @@ -581,12 +581,12 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt } Preconditions.checkNotNull(rollupIndexStorageType); // set rollup index meta to olap table - List rollupColumns = stateMgr.getRollupHandler().checkAndPrepareMaterializedView(addRollupClause, - table, baseRollupIndex); + List rollupColumns = GlobalStateMgr.getCurrentState() + .getRollupHandler().checkAndPrepareMaterializedView(addRollupClause, table, baseRollupIndex); short rollupShortKeyColumnCount = GlobalStateMgr.calcShortKeyColumnCount(rollupColumns, addRollupClause.getProperties()); int rollupSchemaHash = Util.schemaHash(schemaVersion, rollupColumns, bfColumns, bfFpp); - long rollupIndexId = metastore.getNextId(); + long rollupIndexId = GlobalStateMgr.getCurrentState().getNextId(); table.setIndexMeta(rollupIndexId, addRollupClause.getRollupName(), rollupColumns, schemaVersion, rollupSchemaHash, rollupShortKeyColumnCount, rollupIndexStorageType, keysType); } @@ -655,9 +655,12 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt // this is a 1-level partitioned table, use table name as partition name long partitionId = partitionNameToId.get(tableName); - Partition partition = metastore.createPartition(db, table, partitionId, tableName, version, tabletIdSet, - warehouseId); - metastore.buildPartitions(db, table, partition.getSubPartitions().stream().collect(Collectors.toList()), + Partition partition = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .createPartition(db, table, partitionId, tableName, version, tabletIdSet, warehouseId); + GlobalStateMgr.getCurrentState().getStarRocksMeta().buildPartitions( + db, + table, + partition.getSubPartitions().stream().collect(Collectors.toList()), warehouseId); table.addPartition(partition); } else if (partitionInfo.isRangePartition() || partitionInfo.getType() == PartitionType.LIST) { @@ -693,12 +696,14 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt // this is a 2-level partitioned tables List partitions = new ArrayList<>(partitionNameToId.size()); for (Map.Entry entry : partitionNameToId.entrySet()) { - Partition partition = metastore.createPartition(db, table, entry.getValue(), entry.getKey(), version, + Partition partition = GlobalStateMgr.getCurrentState().getStarRocksMeta() + .createPartition(db, table, entry.getValue(), entry.getKey(), version, tabletIdSet, warehouseId); partitions.add(partition); } // It's ok if partitions is empty. - metastore.buildPartitions(db, table, partitions.stream().map(Partition::getSubPartitions) + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .buildPartitions(db, table, partitions.stream().map(Partition::getSubPartitions) .flatMap(p -> p.stream()).collect(Collectors.toList()), warehouseId); for (Partition partition : partitions) { table.addPartition(partition); diff --git a/fe/fe-core/src/main/java/com/starrocks/server/SharedDataStorageVolumeMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/SharedDataStorageVolumeMgr.java index e42a83cd6299c..985895dd91b07 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/SharedDataStorageVolumeMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/SharedDataStorageVolumeMgr.java @@ -325,10 +325,10 @@ protected List> getBindingsOfBuiltinStorageVolume() { List> bindings = new ArrayList<>(); List tableBindings = new ArrayList<>(); List dbBindings = new ArrayList<>(); - List dbIds = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIdsIncludeRecycleBin().stream() + List dbIds = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIdsIncludeRecycleBin().stream() .filter(dbid -> dbid > NEXT_ID_INIT_VALUE).collect(Collectors.toList()); for (Long dbId : dbIds) { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDbIncludeRecycleBin(dbId); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDbIncludeRecycleBin(dbId); Locker locker = new Locker(); locker.lockDatabase(db.getId(), LockType.READ); if (dbToStorageVolume.containsKey(dbId)) { @@ -336,7 +336,7 @@ protected List> getBindingsOfBuiltinStorageVolume() { } dbBindings.add(dbId); try { - List
tables = GlobalStateMgr.getCurrentState().getLocalMetastore().getTablesIncludeRecycleBin(db); + List
tables = GlobalStateMgr.getCurrentState().getStarRocksMeta().getTablesIncludeRecycleBin(db); for (Table table : tables) { Long tableId = table.getId(); if (!tableToStorageVolume.containsKey(tableId) && table.isCloudNativeTableOrMaterializedView()) { diff --git a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java index 3f2395790c1c6..ec5b376c5817f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/com/starrocks/service/FrontendServiceImpl.java @@ -600,7 +600,7 @@ public TListPipesResult listPipes(TListPipesParams params) throws TException { Map pipes = pm.getPipesUnlock(); TListPipesResult result = new TListPipesResult(); for (Pipe pipe : pipes.values()) { - String databaseName = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(pipe.getPipeId().getDbId()) + String databaseName = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(pipe.getPipeId().getDbId()) .map(Database::getOriginName) .orElse(null); @@ -648,7 +648,7 @@ public TListPipeFilesResult listPipeFiles(TListPipeFilesParams params) throws TE file.setPipe_id(record.pipeId); file.setDatabase_name( mayPipe.flatMap(p -> - GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(p.getDbAndName().first) + GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(p.getDbAndName().first) .map(Database::getOriginName)) .orElse("")); file.setPipe_name(mayPipe.map(Pipe::getName).orElse("")); @@ -1965,7 +1965,7 @@ public synchronized TImmutablePartitionResult updateImmutablePartitionInternal(T locker.unLockDatabase(db.getId(), LockType.READ); } if (mutablePartitions.size() <= 0) { - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .addSubPartitions(db, olapTable, partition, 1, warehouseId); } } @@ -2236,7 +2236,7 @@ private static TCreatePartitionResult createPartitionProcess(TCreatePartitionReq try { if (olapTable.getState() == OlapTable.OlapTableState.ROLLUP) { LOG.info("cancel rollup for automatic create partition txn_id={}", request.getTxn_id()); - state.getLocalMetastore().cancelAlter( + state.getAlterJobMgr().cancelAlter( new CancelAlterTableStmt( ShowAlterStmt.AlterType.ROLLUP, new TableName(db.getFullName(), olapTable.getName())), @@ -2245,7 +2245,7 @@ private static TCreatePartitionResult createPartitionProcess(TCreatePartitionReq if (olapTable.getState() == OlapTable.OlapTableState.SCHEMA_CHANGE) { LOG.info("cancel schema change for automatic create partition txn_id={}", request.getTxn_id()); - state.getLocalMetastore().cancelAlter( + state.getAlterJobMgr().cancelAlter( new CancelAlterTableStmt( ShowAlterStmt.AlterType.COLUMN, new TableName(db.getFullName(), olapTable.getName())), @@ -2265,7 +2265,7 @@ private static TCreatePartitionResult createPartitionProcess(TCreatePartitionReq AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(olapTable); analyzer.analyze(ctx, addPartitionClause); } - state.getLocalMetastore().addPartitions(ctx, db, olapTable.getName(), addPartitionClause); + state.getStarRocksMeta().addPartitions(ctx, db, olapTable.getName(), addPartitionClause); } catch (Exception e) { LOG.warn("failed to cancel alter operation", e); errorStatus.setError_msgs(Lists.newArrayList( diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/pipe/ShowPipeStmt.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/pipe/ShowPipeStmt.java index 388185a511012..ed023666b1a1e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/ast/pipe/ShowPipeStmt.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/pipe/ShowPipeStmt.java @@ -69,7 +69,7 @@ public ShowPipeStmt(String dbName, String like, Expr where, List * NOTE: Must be consistent with the META_DATA */ public static void handleShow(List row, Pipe pipe) { - Optional db = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(pipe.getPipeId().getDbId()); + Optional db = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(pipe.getPipeId().getDbId()); row.add(db.map(Database::getFullName).orElse("")); row.add(String.valueOf(pipe.getPipeId().getId())); row.add(pipe.getName()); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/function/MetaFunctions.java b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/function/MetaFunctions.java index 4834bfa05e10f..34933a1185eb1 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/function/MetaFunctions.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/optimizer/function/MetaFunctions.java @@ -101,9 +101,9 @@ public static Table inspectExternalTable(TableName tableName) { } public static Pair inspectTable(TableName tableName) { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(tableName.getDb()) + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(tableName.getDb()) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_BAD_DB_ERROR, tableName.getDb())); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetTable(tableName.getDb(), tableName.getTbl()) + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetTable(tableName.getDb(), tableName.getTbl()) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName)); ConnectContext connectContext = ConnectContext.get(); try { @@ -167,7 +167,7 @@ public static ConstantOperator inspectRelatedMv(ConstantOperator name) { Optional mayDb; Table table = inspectExternalTable(tableName); if (table.isNativeTableOrMaterializedView()) { - mayDb = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(tableName.getDb()); + mayDb = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(tableName.getDb()); } else { mayDb = Optional.empty(); } @@ -179,7 +179,7 @@ public static ConstantOperator inspectRelatedMv(ConstantOperator name) { Set relatedMvs = table.getRelatedMaterializedViews(); JsonArray array = new JsonArray(); for (MvId mv : SetUtils.emptyIfNull(relatedMvs)) { - String mvName = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetTable(mv.getDbId(), mv.getId()) + String mvName = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetTable(mv.getDbId(), mv.getId()) .map(Table::getName) .orElse(null); JsonObject obj = new JsonObject(); @@ -246,7 +246,7 @@ public static ConstantOperator inspectAllPipes() { ConnectContext connectContext = ConnectContext.get(); authOperatorPrivilege(); String currentDb = connectContext.getDatabase(); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().mayGetDb(connectContext.getDatabase()) + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().mayGetDb(connectContext.getDatabase()) .orElseThrow(() -> ErrorReport.buildSemanticException(ErrorCode.ERR_BAD_DB_ERROR, currentDb)); String json = GlobalStateMgr.getCurrentState().getPipeManager().getPipesOfDb(db.getId()); return ConstantOperator.createVarchar(json); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java index 20a6fff10d70e..14939ebfb768f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/plan/PlanFragmentBuilder.java @@ -117,7 +117,6 @@ import com.starrocks.qe.ConnectContext; import com.starrocks.qe.SessionVariable; import com.starrocks.server.GlobalStateMgr; -import com.starrocks.server.LocalMetastore; import com.starrocks.server.RunMode; import com.starrocks.service.FrontendOptions; import com.starrocks.sql.analyzer.DecimalV3FunctionAnalyzer; @@ -220,6 +219,7 @@ import java.util.stream.Collectors; import static com.starrocks.catalog.Function.CompareMode.IS_NONSTRICT_SUPERTYPE_OF; +import static com.starrocks.meta.StarRocksMeta.buildPartitionInfo; import static com.starrocks.sql.common.ErrorType.INTERNAL_ERROR; import static com.starrocks.sql.common.UnsupportedException.unsupportedException; import static com.starrocks.sql.optimizer.operator.scalar.ScalarOperator.isColumnEqualConstant; @@ -265,7 +265,7 @@ public static ExecPlan createPhysicalPlanForMV(ConnectContext connectContext, Collections.reverse(execPlan.getFragments()); // Create a fake table sink here, replaced it after created the MV - PartitionInfo partitionInfo = LocalMetastore.buildPartitionInfo(createStmt); + PartitionInfo partitionInfo = buildPartitionInfo(createStmt); long mvId = GlobalStateMgr.getCurrentState().getNextId(); long dbId = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(createStmt.getTableName().getDb()).getId(); MaterializedView view = GlobalStateMgr.getCurrentState().getMaterializedViewMgr() @@ -870,8 +870,13 @@ public PlanFragment visitPhysicalOlapScan(OptExpression optExpr, ExecPlan contex tabletId2BucketSeq.put(allTabletIds.get(i), i); } scanNode.setTabletId2BucketSeq(tabletId2BucketSeq); - List tablets = - selectTabletIds.stream().map(selectedTable::getTablet).collect(Collectors.toList()); + + List tablets = new ArrayList<>(); + for (Long selectTabletId : selectTabletIds) { + Tablet tablet = GlobalStateMgr.getCurrentState().getTabletMetastore().getTablet( + selectedTable, selectTabletId); + tablets.add(tablet); + } scanNode.addScanRangeLocations(partition, physicalPartition, selectedTable, tablets, localBeId); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticUtils.java b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticUtils.java index 430c31ba04c9e..464e6bd6506a3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticUtils.java +++ b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticUtils.java @@ -283,9 +283,11 @@ public static boolean checkStatisticTableStateNormal() { // check replicate miss for (Partition partition : table.getPartitions()) { - if (partition.getBaseIndex().getTablets().stream() - .anyMatch(t -> ((LocalTablet) t).getNormalReplicaBackendIds().isEmpty())) { - return false; + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + if (physicalPartition.getBaseIndex().getTablets().stream() + .anyMatch(t -> ((LocalTablet) t).getNormalReplicaBackendIds().isEmpty())) { + return false; + } } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticsMetaManager.java b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticsMetaManager.java index e62b42d68f429..adce6fc2590a5 100644 --- a/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticsMetaManager.java +++ b/fe/fe-core/src/main/java/com/starrocks/statistic/StatisticsMetaManager.java @@ -23,6 +23,7 @@ import com.starrocks.catalog.LocalTablet; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PhysicalPartition; import com.starrocks.common.Config; import com.starrocks.common.DdlException; import com.starrocks.common.Pair; @@ -68,7 +69,7 @@ private boolean createDatabase() { LOG.info("create statistics db start"); CreateDbStmt dbStmt = new CreateDbStmt(false, StatsConstants.STATISTICS_DB_NAME); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().createDb(dbStmt.getFullDbName()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createDb(dbStmt.getFullDbName()); } catch (UserException e) { LOG.warn("Failed to create database ", e); return false; @@ -103,10 +104,12 @@ private boolean checkReplicateNormal(String tableName) { boolean check = true; for (Partition partition : table.getPartitions()) { // check replicate miss - if (partition.getBaseIndex().getTablets().stream() - .anyMatch(t -> ((LocalTablet) t).getNormalReplicaBackendIds().isEmpty())) { - check = false; - break; + for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { + if (physicalPartition.getBaseIndex().getTablets().stream() + .anyMatch(t -> ((LocalTablet) t).getNormalReplicaBackendIds().isEmpty())) { + check = false; + break; + } } } @@ -160,7 +163,7 @@ private boolean createSampleStatisticsTable(ConnectContext context) { ""); Analyzer.analyze(stmt, context); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(stmt); } catch (UserException e) { LOG.warn("Failed to create sample statistics, ", e); return false; @@ -192,7 +195,7 @@ private boolean createFullStatisticsTable(ConnectContext context) { ""); Analyzer.analyze(stmt, context); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(stmt); } catch (UserException e) { LOG.warn("Failed to create full statistics table", e); return false; @@ -223,7 +226,7 @@ private boolean createHistogramStatisticsTable(ConnectContext context) { ""); Analyzer.analyze(stmt, context); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(stmt); } catch (UserException e) { LOG.warn("Failed to create histogram statistics table", e); return false; @@ -261,7 +264,7 @@ private boolean createExternalFullStatisticsTable(ConnectContext context) { ""); Analyzer.analyze(stmt, context); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(stmt); } catch (UserException e) { LOG.warn("Failed to create full statistics table", e); return false; @@ -291,7 +294,7 @@ private boolean createExternalHistogramStatisticsTable(ConnectContext context) { ""); Analyzer.analyze(stmt, context); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(stmt); } catch (UserException e) { LOG.warn("Failed to create external histogram statistics table", e); return false; @@ -330,7 +333,7 @@ private boolean dropTable(String tableName) { new TableName(StatsConstants.STATISTICS_DB_NAME, tableName), true); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(stmt); } catch (DdlException e) { LOG.warn("Failed to drop table", e); return false; diff --git a/fe/fe-core/src/main/java/com/starrocks/task/AlterReplicaTask.java b/fe/fe-core/src/main/java/com/starrocks/task/AlterReplicaTask.java index f76cc401e7053..b6fdbb7c4dca6 100644 --- a/fe/fe-core/src/main/java/com/starrocks/task/AlterReplicaTask.java +++ b/fe/fe-core/src/main/java/com/starrocks/task/AlterReplicaTask.java @@ -53,6 +53,7 @@ import com.starrocks.common.util.TimeUtils; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.meta.TabletMetastore; import com.starrocks.persist.ReplicaPersistInfo; import com.starrocks.server.GlobalStateMgr; import com.starrocks.thrift.TAlterJobType; @@ -301,6 +302,8 @@ public TAlterTabletReqV2 toThrift() { * And because alter request report success, it means that we can increase replica's version to X. */ public void handleFinishAlterTask() throws Exception { + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(getDbId()); if (db == null) { throw new MetaNotFoundException("database " + getDbId() + " does not exist"); @@ -322,7 +325,7 @@ public void handleFinishAlterTask() throws Exception { if (index == null) { throw new MetaNotFoundException("index " + getIndexId() + " does not exist"); } - Tablet tablet = index.getTablet(getTabletId()); + Tablet tablet = tabletMetastore.getTablet(index, getTabletId()); Preconditions.checkNotNull(tablet, getTabletId()); if (!tbl.isCloudNativeTableOrMaterializedView()) { Replica replica = ((LocalTablet) tablet).getReplicaById(getNewReplicaId()); diff --git a/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java b/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java index e2994e93aeb8c..f2b06cfc3f588 100644 --- a/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/task/TabletTaskExecutor.java @@ -378,7 +378,6 @@ public static int countMaxTasksPerBackend(List tasks) { public static void deleteAllReplicas(OlapTable olapTable) { HashMap batchTaskMap = new HashMap<>(); - // drop all replicas for (Partition partition : olapTable.getAllPartitions()) { for (PhysicalPartition physicalPartition : partition.getSubPartitions()) { @@ -387,10 +386,12 @@ public static void deleteAllReplicas(OlapTable olapTable) { for (MaterializedIndex materializedIndex : allIndices) { long indexId = materializedIndex.getId(); int schemaHash = olapTable.getSchemaHashByIndexId(indexId); - for (Tablet tablet : materializedIndex.getTablets()) { + List tabletList = GlobalStateMgr.getCurrentState().getTabletMetastore() + .getAllTablets(materializedIndex); + for (Tablet tablet : tabletList) { long tabletId = tablet.getId(); - List replicas = ((LocalTablet) tablet).getImmutableReplicas(); - for (Replica replica : replicas) { + List replicaList = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllReplicas(tablet); + for (Replica replica : replicaList) { long backendId = replica.getBackendId(); DropReplicaTask dropTask = new DropReplicaTask(backendId, tabletId, schemaHash, true); AgentBatchTask batchTask = batchTaskMap.get(backendId); @@ -402,7 +403,7 @@ public static void deleteAllReplicas(OlapTable olapTable) { LOG.info("delete tablet[{}] from backend[{}] because table {}-{} is dropped", tabletId, backendId, olapTable.getId(), olapTable.getName()); } // end for replicas - } // end for tablets + } } } // end for indices } // end for partitions diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java index fba3791c29520..b8278eee8cd98 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/LakeTableTxnStateListener.java @@ -145,9 +145,9 @@ public void preCommit(TransactionState txnState, List finished PhysicalPartition partition = table.getPhysicalPartition(partitionId); List allIndices = txnState.getPartitionLoadedTblIndexes(table.getId(), partition); for (MaterializedIndex index : allIndices) { - Optional unfinishedTablet = - index.getTablets().stream().filter(t -> !finishedTabletsOfThisTable.contains(t.getId())) - .findAny(); + List tabletList = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index); + Optional unfinishedTablet = tabletList.stream().filter( + t -> !finishedTabletsOfThisTable.contains(t.getId())).findAny(); if (!unfinishedTablet.isPresent()) { continue; } diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java index 49d502ddbb76e..fe0157e9b6f4e 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/OlapTableTxnStateListener.java @@ -180,7 +180,9 @@ public void preCommit(TransactionState txnState, List tabletCo List allIndices = txnState.getPartitionLoadedTblIndexes(table.getId(), partition); int quorumReplicaNum = table.getPartitionInfo().getQuorumNum(partition.getParentId(), table.writeQuorum()); for (MaterializedIndex index : allIndices) { - for (Tablet tablet : index.getTablets()) { + List tabletList = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index); + + for (Tablet tablet : tabletList) { long tabletId = tablet.getId(); Set commitBackends = tabletToBackends.get(tabletId); diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java b/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java index ffd8543b10af0..ae738f97a80c8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/PublishVersionDaemon.java @@ -52,6 +52,7 @@ import com.starrocks.lake.TxnInfoHelper; import com.starrocks.lake.Utils; import com.starrocks.lake.compaction.Quantiles; +import com.starrocks.meta.TabletMetastore; import com.starrocks.proto.DeleteTxnLogRequest; import com.starrocks.proto.TxnInfoPB; import com.starrocks.rpc.BrpcProxy; @@ -509,16 +510,20 @@ public boolean publishPartitionBatch(Database db, long tableId, PartitionPublish txnState.getTransactionId()); continue; } + + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); + List tablets = tabletMetastore.getAllTablets(index); + if (index.getState() == MaterializedIndex.IndexState.SHADOW) { if (shadowTabletsMap.containsKey(versions.get(i))) { - shadowTabletsMap.get(versions.get(i)).addAll(index.getTablets()); + shadowTabletsMap.get(versions.get(i)).addAll(tablets); } else { - Set tabletsNew = new HashSet<>(index.getTablets()); + Set tabletsNew = new HashSet<>(tablets); shadowTabletsMap.put(versions.get(i), tabletsNew); } } else { normalTablets = (normalTablets == null) ? Sets.newHashSet() : normalTablets; - normalTablets.addAll(index.getTablets()); + normalTablets.addAll(tablets); } } } @@ -791,12 +796,15 @@ private boolean publishPartition(@NotNull Database db, @NotNull TableCommitInfo LOG.info("Ignored index {} for transaction {}", table.getIndexNameById(index.getId()), txnId); continue; } + + List tabletList = GlobalStateMgr.getCurrentState().getTabletMetastore().getAllTablets(index); + if (index.getState() == MaterializedIndex.IndexState.SHADOW) { shadowTablets = (shadowTablets == null) ? Lists.newArrayList() : shadowTablets; - shadowTablets.addAll(index.getTablets()); + shadowTablets.addAll(tabletList); } else { normalTablets = (normalTablets == null) ? Lists.newArrayList() : normalTablets; - normalTablets.addAll(index.getTablets()); + normalTablets.addAll(tabletList); } } } finally { diff --git a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java index 3bb98a819381d..05a78f92c2004 100644 --- a/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java +++ b/fe/fe-core/src/main/java/com/starrocks/transaction/TransactionChecker.java @@ -23,6 +23,7 @@ import com.starrocks.catalog.Tablet; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.meta.TabletMetastore; import com.starrocks.server.GlobalStateMgr; import java.util.ArrayList; @@ -99,7 +100,7 @@ public static TransactionChecker create(TransactionState txn, Database db) { List partitions = new ArrayList<>(); for (TableCommitInfo tableCommitInfo : txn.getIdToTableCommitInfos().values()) { OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore() - .getTable(db.getId(), tableCommitInfo.getTableId()); + .getTable(db.getId(), tableCommitInfo.getTableId()); if (table == null || table.isCloudNativeTableOrMaterializedView()) { continue; } @@ -119,7 +120,9 @@ public static TransactionChecker create(TransactionState txn, Database db) { List allIndices = txn.getPartitionLoadedTblIndexes(tableCommitInfo.getTableId(), partition); for (MaterializedIndex index : allIndices) { - for (Tablet tablet : index.getTablets()) { + TabletMetastore tabletMetastore = GlobalStateMgr.getCurrentState().getTabletMetastore(); + List tablets = tabletMetastore.getAllTablets(index); + for (Tablet tablet : tablets) { partitionChecker.tablets.add((LocalTablet) tablet); } } diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/AlterJobV2Test.java b/fe/fe-core/src/test/java/com/starrocks/alter/AlterJobV2Test.java index 3f896604efd7d..e9ac3d23bf637 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/AlterJobV2Test.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/AlterJobV2Test.java @@ -187,7 +187,7 @@ public void testModifyRelatedColumnWithMv() { String sql = "CREATE MATERIALIZED VIEW test.mv2 DISTRIBUTED BY HASH(k1) " + " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1') AS SELECT k1, k2 FROM modify_column_test"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); // modify column which define in mv @@ -197,7 +197,7 @@ public void testModifyRelatedColumnWithMv() { waitForSchemaChangeAlterJobFinish(); MaterializedView mv2 = - (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test").getTable("mv2"); + (MaterializedView) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test").getTable("mv2"); Assert.assertFalse(mv2.isActive()); } catch (Exception e) { e.printStackTrace(); @@ -215,7 +215,7 @@ public void testModifyWithSelectStarMV1() throws Exception { " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1') " + "AS SELECT * FROM modify_column_test3"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); String alterStmtStr = "alter table test.modify_column_test3 modify column k2 varchar(20)"; @@ -223,7 +223,7 @@ public void testModifyWithSelectStarMV1() throws Exception { DDLStmtExecutor.execute(alterTableStmt, connectContext); waitForSchemaChangeAlterJobFinish(); - MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore() + MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState().getStarRocksMeta() .getDb("test").getTable("mv3"); Assert.assertTrue(!mv.isActive()); } finally { @@ -241,7 +241,7 @@ public void testModifyWithSelectStarMV2() throws Exception { " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1') " + "AS SELECT * FROM testModifyWithSelectStarMV2"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); String alterStmtStr = "alter table test.testModifyWithSelectStarMV2 add column k4 bigint"; @@ -250,7 +250,7 @@ public void testModifyWithSelectStarMV2() throws Exception { waitForSchemaChangeAlterJobFinish(); MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState() - .getLocalMetastore().getDb("test").getTable("mv6"); + .getStarRocksMeta().getDb("test").getTable("mv6"); Assert.assertTrue(mv.isActive()); } catch (Exception e) { e.printStackTrace(); @@ -270,7 +270,7 @@ public void testModifyWithSelectStarMV3() throws Exception { " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1') " + "AS SELECT * FROM modify_column_test5"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); String alterStmtStr = "alter table test.modify_column_test5 drop column k2"; @@ -279,7 +279,7 @@ public void testModifyWithSelectStarMV3() throws Exception { waitForSchemaChangeAlterJobFinish(); MaterializedView mv = (MaterializedView) GlobalStateMgr.getCurrentState() - .getLocalMetastore().getDb("test").getTable("mv5"); + .getStarRocksMeta().getDb("test").getTable("mv5"); Assert.assertTrue(!mv.isActive()); } catch (Exception e) { Assert.fail(); @@ -297,7 +297,7 @@ public void testModifyWithExpr() throws Exception { " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1')" + " AS SELECT k1, k2 + 1 FROM modify_column_test4"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); { @@ -309,7 +309,7 @@ public void testModifyWithExpr() throws Exception { waitForSchemaChangeAlterJobFinish(); MaterializedView mv = (MaterializedView) GlobalStateMgr - .getCurrentState().getLocalMetastore().getDb("test").getTable("mv4"); + .getCurrentState().getStarRocksMeta().getDb("test").getTable("mv4"); Assert.assertTrue(mv.isActive()); } @@ -322,7 +322,7 @@ public void testModifyWithExpr() throws Exception { waitForSchemaChangeAlterJobFinish(); MaterializedView mv = (MaterializedView) GlobalStateMgr - .getCurrentState().getLocalMetastore().getDb("test").getTable("mv4"); + .getCurrentState().getStarRocksMeta().getDb("test").getTable("mv4"); Assert.assertFalse(mv.isActive()); System.out.println(mv.getInactiveReason()); Assert.assertTrue(mv.getInactiveReason().contains("base table schema changed for columns: k2")); @@ -341,7 +341,7 @@ public void testModifyUnRelatedColumnWithMv() { String sql = "CREATE MATERIALIZED VIEW test.mv1 DISTRIBUTED BY HASH(k1) " + " BUCKETS 10 REFRESH ASYNC properties('replication_num' = '1') AS SELECT k1, k2 FROM modify_column_test"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore() + GlobalStateMgr.getCurrentState().getStarRocksMeta() .createMaterializedView((CreateMaterializedViewStatement) statementBase); // modify column which not define in mv @@ -351,7 +351,7 @@ public void testModifyUnRelatedColumnWithMv() { waitForSchemaChangeAlterJobFinish(); MaterializedView mv = - (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test").getTable("mv1"); + (MaterializedView) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test").getTable("mv1"); Assert.assertTrue(mv.isActive()); } catch (Exception e) { e.printStackTrace(); diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java index f9934a38334b9..c8904be16a05b 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTest.java @@ -150,107 +150,107 @@ public static void beforeClass() throws Exception { starRocksAssert = new StarRocksAssert(connectContext); starRocksAssert.withDatabase("test").useDatabase("test") - .withTable("CREATE TABLE test.tbl1\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');") - - .withTable("CREATE TABLE test.tbl2\n" + - "(\n" + - " k1 date,\n" + - " v1 int sum\n" + - ")\n" + - "DISTRIBUTED BY HASH (k1) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');") - - .withTable("CREATE TABLE test.tbl3\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');") - - .withTable("CREATE TABLE test.tbl4\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01'),\n" + - " PARTITION p3 values less than('2020-04-01'),\n" + - " PARTITION p4 values less than('2020-05-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES" + - "(" + - " 'replication_num' = '1',\n" + - " 'in_memory' = 'false',\n" + - " 'storage_medium' = 'SSD',\n" + - " 'storage_cooldown_time' = '9999-12-31 00:00:00'\n" + - ");") - .withTable("CREATE TABLE t_recharge_detail(\n" + - " id bigint ,\n" + - " user_id bigint ,\n" + - " recharge_money decimal(32,2) , \n" + - " province varchar(20) not null,\n" + - " dt varchar(20) not null\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (dt,province) (\n" + - " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\")),\n" + - " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\"))\n" + - ")\n" + - "DISTRIBUTED BY HASH(`id`) BUCKETS 10 \n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\",\n" + - "\"in_memory\" = \"false\"\n" + - ");") - .withTable("CREATE TABLE test.site_access_date_trunc (\n" + - " event_day DATETIME NOT NULL,\n" + - " site_id INT DEFAULT '10',\n" + - " city_code VARCHAR(100),\n" + - " user_name VARCHAR(32) DEFAULT '',\n" + - " pv BIGINT DEFAULT '0'\n" + - ")\n" + - "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + - "PARTITION BY date_trunc('day', event_day)\n" + - "DISTRIBUTED BY HASH(event_day, site_id)\n" + - "PROPERTIES(\n" + - " \"replication_num\" = \"1\"\n" + - ");") - .withTable("CREATE TABLE site_access_time_slice (\n" + - " event_day datetime,\n" + - " site_id INT DEFAULT '10',\n" + - " city_code VARCHAR(100),\n" + - " user_name VARCHAR(32) DEFAULT '',\n" + - " pv BIGINT DEFAULT '0'\n" + - ")\n" + - "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + - "PARTITION BY time_slice(event_day, interval 1 day)\n" + - "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 32\n" + - "PROPERTIES(\n" + - " \"partition_live_number\" = \"3\",\n" + - " \"replication_num\" = \"1\"\n" + - ");"); + .withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');") + + .withTable("CREATE TABLE test.tbl2\n" + + "(\n" + + " k1 date,\n" + + " v1 int sum\n" + + ")\n" + + "DISTRIBUTED BY HASH (k1) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');") + + .withTable("CREATE TABLE test.tbl3\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');") + + .withTable("CREATE TABLE test.tbl4\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01'),\n" + + " PARTITION p3 values less than('2020-04-01'),\n" + + " PARTITION p4 values less than('2020-05-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES" + + "(" + + " 'replication_num' = '1',\n" + + " 'in_memory' = 'false',\n" + + " 'storage_medium' = 'SSD',\n" + + " 'storage_cooldown_time' = '9999-12-31 00:00:00'\n" + + ");") + .withTable("CREATE TABLE t_recharge_detail(\n" + + " id bigint ,\n" + + " user_id bigint ,\n" + + " recharge_money decimal(32,2) , \n" + + " province varchar(20) not null,\n" + + " dt varchar(20) not null\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (dt,province) (\n" + + " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\")),\n" + + " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\"))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 10 \n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"in_memory\" = \"false\"\n" + + ");") + .withTable("CREATE TABLE test.site_access_date_trunc (\n" + + " event_day DATETIME NOT NULL,\n" + + " site_id INT DEFAULT '10',\n" + + " city_code VARCHAR(100),\n" + + " user_name VARCHAR(32) DEFAULT '',\n" + + " pv BIGINT DEFAULT '0'\n" + + ")\n" + + "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + + "PARTITION BY date_trunc('day', event_day)\n" + + "DISTRIBUTED BY HASH(event_day, site_id)\n" + + "PROPERTIES(\n" + + " \"replication_num\" = \"1\"\n" + + ");") + .withTable("CREATE TABLE site_access_time_slice (\n" + + " event_day datetime,\n" + + " site_id INT DEFAULT '10',\n" + + " city_code VARCHAR(100),\n" + + " user_name VARCHAR(32) DEFAULT '',\n" + + " pv BIGINT DEFAULT '0'\n" + + ")\n" + + "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + + "PARTITION BY time_slice(event_day, interval 1 day)\n" + + "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 32\n" + + "PROPERTIES(\n" + + " \"partition_live_number\" = \"3\",\n" + + " \"replication_num\" = \"1\"\n" + + ");"); } @AfterClass @@ -259,7 +259,7 @@ public static void tearDown() throws Exception { String dropSQL = "drop table test_partition_exception"; try { DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } catch (Exception ex) { } @@ -282,21 +282,21 @@ private static void createTable(String sql) throws Exception { private static void createMaterializedView(String sql) throws Exception { CreateMaterializedViewStatement createMaterializedViewStatement = - (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().createMaterializedView(createMaterializedViewStatement); + (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createMaterializedView(createMaterializedViewStatement); } private static void dropMaterializedView(String sql) throws Exception { DropMaterializedViewStmt dropMaterializedViewStmt = - (DropMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropMaterializedView(dropMaterializedViewStmt); + (DropMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropMaterializedView(dropMaterializedViewStmt); } private static void alterMaterializedView(String sql, boolean expectedException) throws Exception { AlterMaterializedViewStmt alterMaterializedViewStmt = - (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); + (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterMaterializedViewStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterMaterializedViewStmt); if (expectedException) { Assert.fail(); } @@ -310,12 +310,12 @@ private static void alterMaterializedView(String sql, boolean expectedException) private static void refreshMaterializedView(String sql) throws Exception { RefreshMaterializedViewStatement refreshMaterializedViewStatement = - (RefreshMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); + (RefreshMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); try { - GlobalStateMgr.getCurrentState().getLocalMetastore() - .refreshMaterializedView(refreshMaterializedViewStatement.getMvName().getDb(), - refreshMaterializedViewStatement.getMvName().getTbl(), false, null, - Constants.TaskRunPriority.LOWEST.value(), false, true); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .refreshMaterializedView(refreshMaterializedViewStatement.getMvName().getDb(), + refreshMaterializedViewStatement.getMvName().getTbl(), false, null, + Constants.TaskRunPriority.LOWEST.value(), false, true); } catch (Exception e) { e.printStackTrace(); Assert.fail(); @@ -324,9 +324,9 @@ private static void refreshMaterializedView(String sql) throws Exception { private static void cancelRefreshMaterializedView(String sql, boolean expectedException) throws Exception { CancelRefreshMaterializedViewStmt cancelRefresh = - (CancelRefreshMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); + (CancelRefreshMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().cancelRefreshMaterializedView(cancelRefresh); + GlobalStateMgr.getCurrentState().getStarRocksMeta().cancelRefreshMaterializedView(cancelRefresh); if (expectedException) { Assert.fail(); } @@ -356,7 +356,7 @@ private static void alterTableWithNewParser(String sql, boolean expectedExceptio private static void alterTableWithNewParserAndExceptionMsg(String sql, String msg) throws Exception { AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(connectContext, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(connectContext, alterTableStmt); } catch (Exception e) { Assert.assertEquals(msg, e.getMessage()); } @@ -365,36 +365,36 @@ private static void alterTableWithNewParserAndExceptionMsg(String sql, String ms @Test public void testRenameMaterializedView() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testTable1\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testTable1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String sql = "create materialized view mv1 " + - "partition by k1 " + - "distributed by hash(k2) " + - "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ") " + - "as select k1, k2 from test.testTable1;"; + "partition by k1 " + + "distributed by hash(k2) " + + "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ") " + + "as select k1, k2 from test.testTable1;"; createMaterializedView(sql); String alterStmt = "alter materialized view test.mv1 rename mv2"; alterMaterializedView(alterStmt, false); - MaterializedView materializedView = (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore(). - getDb("test").getTable("mv2"); + MaterializedView materializedView = (MaterializedView) GlobalStateMgr.getCurrentState().getStarRocksMeta(). + getDb("test").getTable("mv2"); TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); Task task = taskManager.getTask(TaskBuilder.getMvTaskName(materializedView.getId())); Assert.assertEquals("insert overwrite `mv2` SELECT `test`.`testTable1`.`k1`, `test`.`testTable1`.`k2`\n" + - "FROM `test`.`testTable1`", task.getDefinition()); + "FROM `test`.`testTable1`", task.getDefinition()); ConnectContext.get().setCurrentUserIdentity(UserIdentity.ROOT); ConnectContext.get().setCurrentRoleIds(UserIdentity.ROOT); dropMaterializedView("drop materialized view test.mv2"); @@ -403,45 +403,46 @@ public void testRenameMaterializedView() throws Exception { @Test public void testCouldNotFindMaterializedView() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testTable1\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');") - .withTable("CREATE TABLE test.testTable2\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testTable1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');") + .withTable("CREATE TABLE test.testTable2\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String sql = "create materialized view mv1 " + - "partition by k1 " + - "distributed by hash(k2) " + - "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ") " + - "as select k1, k2 from test.testTable1;"; + "partition by k1 " + + "distributed by hash(k2) " + + "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ") " + + "as select k1, k2 from test.testTable1;"; createMaterializedView(sql); starRocksAssert.getCtx().setCurrentRoleIds(GlobalStateMgr.getCurrentState().getAuthorizationMgr().getRoleIdsByUser( - starRocksAssert.getCtx().getCurrentUserIdentity())); + starRocksAssert.getCtx().getCurrentUserIdentity())); dropMaterializedView("drop materialized view test.mv1"); - OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test").getTable("testTable1"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getDb("test").getTable("testTable1"); // this for mock olapTable.getIndexNameById(mvIdx.getId()) == Null table.deleteIndexInfo("testTable1"); try { @@ -455,19 +456,19 @@ public void testCouldNotFindMaterializedView() throws Exception { @Test public void testRenameTable() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testRenameTable1\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testRenameTable1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String alterStmt = "alter table test.testRenameTable1 rename testRenameTable2"; alterTableWithNewParser(alterStmt, false); } @@ -475,27 +476,27 @@ public void testRenameTable() throws Exception { @Test public void testChangeMaterializedViewRefreshScheme() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testTable2\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testTable2\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String sql = "create materialized view mv1 " + - "partition by k1 " + - "distributed by hash(k2) " + - "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ") " + - "as select k1, k2 from test.testTable2;"; + "partition by k1 " + + "distributed by hash(k2) " + + "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ") " + + "as select k1, k2 from test.testTable2;"; createMaterializedView(sql); String alterStmt = "alter materialized view mv1 refresh async EVERY(INTERVAL 1 minute)"; alterMaterializedView(alterStmt, false); @@ -509,61 +510,61 @@ public void testChangeMaterializedViewRefreshScheme() throws Exception { @Test public void testRefreshMaterializedView() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testTable3\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testTable3\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String sql = "create materialized view mv1 " + - "partition by k1 " + - "distributed by hash(k2) " + - "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ") " + - "as select k1, k2 from test.testTable3;"; + "partition by k1 " + + "distributed by hash(k2) " + + "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ") " + + "as select k1, k2 from test.testTable3;"; createMaterializedView(sql); String alterStmt = "refresh materialized view test.mv1"; refreshMaterializedView(alterStmt); starRocksAssert.getCtx().setCurrentRoleIds(GlobalStateMgr.getCurrentState().getAuthorizationMgr().getRoleIdsByUser( - starRocksAssert.getCtx().getCurrentUserIdentity())); + starRocksAssert.getCtx().getCurrentUserIdentity())); dropMaterializedView("drop materialized view test.mv1"); } @Test public void testCancelRefreshMaterializedView() throws Exception { starRocksAssert.useDatabase("test") - .withTable("CREATE TABLE test.testTable4\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - " PARTITION p1 values less than('2020-02-01'),\n" + - " PARTITION p2 values less than('2020-03-01')\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.testTable4\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values less than('2020-02-01'),\n" + + " PARTITION p2 values less than('2020-03-01')\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String sql = "create materialized view mv1 " + - "partition by k1 " + - "distributed by hash(k2) " + - "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ") " + - "as select k1, k2 from test.testTable4;"; + "partition by k1 " + + "distributed by hash(k2) " + + "refresh async START('2122-12-31') EVERY(INTERVAL 1 HOUR) " + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ") " + + "as select k1, k2 from test.testTable4;"; starRocksAssert.getCtx().setCurrentRoleIds(GlobalStateMgr.getCurrentState().getAuthorizationMgr().getRoleIdsByUser( - starRocksAssert.getCtx().getCurrentUserIdentity())); + starRocksAssert.getCtx().getCurrentUserIdentity())); createMaterializedView(sql); String alterStmt = "refresh materialized view test.mv1"; refreshMaterializedView(alterStmt); @@ -573,12 +574,12 @@ public void testCancelRefreshMaterializedView() throws Exception { @Test public void testConflictAlterOperations() throws Exception { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl1"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "tbl1"); String stmt = - "alter table test.tbl1 add partition p3 values less than('2020-04-01'), " + - "add partition p4 values less than('2020-05-01')"; + "alter table test.tbl1 add partition p3 values less than('2020-04-01'), " + + "add partition p4 values less than('2020-05-01')"; alterTableWithNewParser(stmt, true); stmt = "alter table test.tbl1 add partition p3 values less than('2020-04-01'), drop partition p4"; @@ -606,12 +607,12 @@ public void testConflictAlterOperations() throws Exception { // enable dynamic partition // not adding the `start` property so that it won't drop the origin partition p1, p2 and p3 stmt = "alter table test.tbl1 set (\n" + - "'dynamic_partition.enable' = 'true',\n" + - "'dynamic_partition.time_unit' = 'DAY',\n" + - "'dynamic_partition.end' = '3',\n" + - "'dynamic_partition.prefix' = 'p',\n" + - "'dynamic_partition.buckets' = '3'\n" + - " );"; + "'dynamic_partition.enable' = 'true',\n" + + "'dynamic_partition.time_unit' = 'DAY',\n" + + "'dynamic_partition.end' = '3',\n" + + "'dynamic_partition.prefix' = 'p',\n" + + "'dynamic_partition.buckets' = '3'\n" + + " );"; alterTableWithNewParser(stmt, false); Assert.assertTrue(tbl.getTableProperty().getDynamicPartitionProperty().isEnabled()); @@ -619,12 +620,12 @@ public void testConflictAlterOperations() throws Exception { // add partition when dynamic partition is enable stmt = "alter table test.tbl1 add partition p3 values less than('2020-04-01') " + - "distributed by hash(k2) buckets 4 PROPERTIES ('replication_num' = '1')"; + "distributed by hash(k2) buckets 4 PROPERTIES ('replication_num' = '1')"; alterTableWithNewParser(stmt, true); // add temp partition when dynamic partition is enable stmt = "alter table test.tbl1 add temporary partition tp3 values less than('2020-04-01') " + - "distributed by hash(k2) buckets 4 PROPERTIES ('replication_num' = '1')"; + "distributed by hash(k2) buckets 4 PROPERTIES ('replication_num' = '1')"; alterTableWithNewParser(stmt, false); Assert.assertEquals(1, tbl.getTempPartitions().size()); @@ -635,7 +636,7 @@ public void testConflictAlterOperations() throws Exception { // add partition when dynamic partition is disable stmt = "alter table test.tbl1 add partition p3 values less than('2020-04-01') " + - "distributed by hash(k2) buckets 4"; + "distributed by hash(k2) buckets 4"; alterTableWithNewParser(stmt, false); // set table's default replication num @@ -652,25 +653,25 @@ 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) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl2"); + OlapTable tbl2 = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "tbl2"); Partition partition = tbl2.getPartition(tbl2.getName()); Assert.assertEquals(Short.valueOf("1"), - Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); // partition replication num and table default replication num are updated at the same time in unpartitioned table stmt = "alter table test.tbl2 set ('replication_num' = '3');"; alterTableWithNewParser(stmt, false); Assert.assertEquals(Short.valueOf("3"), - Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); Assert.assertEquals(Short.valueOf("3"), tbl2.getDefaultReplicationNum()); stmt = "alter table test.tbl2 set ('default.replication_num' = '2');"; alterTableWithNewParser(stmt, false); Assert.assertEquals(Short.valueOf("2"), - Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); Assert.assertEquals(Short.valueOf("2"), tbl2.getDefaultReplicationNum()); stmt = "alter table test.tbl2 modify partition tbl2 set ('replication_num' = '1');"; alterTableWithNewParser(stmt, false); Assert.assertEquals(Short.valueOf("1"), - Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId()))); Assert.assertEquals(Short.valueOf("1"), tbl2.getDefaultReplicationNum()); Thread.sleep(5000); // sleep to wait dynamic partition scheduler run @@ -683,8 +684,8 @@ public void testConflictAlterOperations() throws Exception { alterTableWithNewParser(stmt, false); stmt = "alter table test.tbl1 " + - "add TEMPORARY partition p5 values [('2020-04-10'), ('2020-05-10')) ('replication_num' = '1') " + - "DISTRIBUTED BY HASH(k2) BUCKETS 3 PROPERTIES('replication_num' = '1');"; + "add TEMPORARY partition p5 values [('2020-04-10'), ('2020-05-10')) ('replication_num' = '1') " + + "DISTRIBUTED BY HASH(k2) BUCKETS 3 PROPERTIES('replication_num' = '1');"; alterTableWithNewParser(stmt, false); //rename table stmt = "alter table test.tbl1 rename newTableName"; @@ -694,8 +695,8 @@ public void testConflictAlterOperations() throws Exception { // test batch update range partitions' properties @Test public void testBatchUpdatePartitionProperties() throws Exception { - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable tbl4 = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl4"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable tbl4 = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "tbl4"); Partition p1 = tbl4.getPartition("p1"); Partition p2 = tbl4.getPartition("p2"); Partition p3 = tbl4.getPartition("p3"); @@ -706,12 +707,12 @@ public void testBatchUpdatePartitionProperties() throws Exception { List partitionList = Lists.newArrayList(p1, p2, p4); for (Partition partition : partitionList) { Assert.assertEquals(Short.valueOf("1"), - Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); } alterTableWithNewParser(stmt, false); for (Partition partition : partitionList) { Assert.assertEquals(Short.valueOf("3"), - Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); } Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(p3.getId()))); @@ -749,7 +750,7 @@ public void testBatchUpdatePartitionProperties() throws Exception { alterTableWithNewParser(stmt, false); for (Partition partition : partitionList) { Assert.assertEquals(Short.valueOf("1"), - Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); + Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId()))); } } @@ -769,8 +770,8 @@ public void testDynamicPartitionDropAndAdd() throws Exception { alterTable(stmt, false); Thread.sleep(5000); // sleep to wait dynamic partition scheduler run - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl3"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksConnector().getDb("test"); + OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksConnector().getTable(db.getFullName(), "tbl3"); Assert.assertEquals(4, tbl.getPartitionNames().size()); Assert.assertNull(tbl.getPartition("p1")); Assert.assertNull(tbl.getPartition("p2")); @@ -784,11 +785,11 @@ private void waitSchemaChangeJobDone(boolean rollupJob, OlapTable tb) throws Int for (AlterJobV2 alterJobV2 : alterJobs.values()) { while (!alterJobV2.getJobState().isFinalState()) { System.out.println( - "alter job " + alterJobV2.getJobId() + " is running. state: " + alterJobV2.getJobState()); + "alter job " + alterJobV2.getJobId() + " is running. state: " + alterJobV2.getJobState()); Thread.sleep(1000); } System.out.println(alterJobV2.getType() + " alter job " + alterJobV2.getJobId() + " is done. state: " + - alterJobV2.getJobState()); + alterJobV2.getJobState()); Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState()); } checkTableStateToNormal(tb); @@ -798,36 +799,36 @@ private void waitSchemaChangeJobDone(boolean rollupJob, OlapTable tb) throws Int public void testSetDynamicPropertiesInNormalTable() throws Exception { String tableName = "no_dynamic_table"; String createOlapTblStmt = "CREATE TABLE test.`" + tableName + "` (\n" + - " `k1` date NULL COMMENT \"\",\n" + - " `k2` int NULL COMMENT \"\",\n" + - " `k3` smallint NULL COMMENT \"\",\n" + - " `v1` varchar(2048) NULL COMMENT \"\",\n" + - " `v2` datetime NULL COMMENT \"\"\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + - "COMMENT \"OLAP\"\n" + - "PARTITION BY RANGE (k1)\n" + - "(\n" + - "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + - "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + - "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + - ")\n" + - "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ");"; + " `k1` date NULL COMMENT \"\",\n" + + " `k2` int NULL COMMENT \"\",\n" + + " `k3` smallint NULL COMMENT \"\",\n" + + " `v1` varchar(2048) NULL COMMENT \"\",\n" + + " `v2` datetime NULL COMMENT \"\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + + "COMMENT \"OLAP\"\n" + + "PARTITION BY RANGE (k1)\n" + + "(\n" + + "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + + "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + + "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");"; createTable(createOlapTblStmt); String alterStmt = "alter table test." + tableName + " set (\"dynamic_partition.enable\" = \"true\");"; alterTableWithNewParserAndExceptionMsg(alterStmt, "Table test.no_dynamic_table is not a dynamic partition table."); // test set dynamic properties in a no dynamic partition table String stmt = "alter table test." + tableName + " set (\n" + - "'dynamic_partition.enable' = 'true',\n" + - "'dynamic_partition.time_unit' = 'DAY',\n" + - "'dynamic_partition.start' = '-3',\n" + - "'dynamic_partition.end' = '3',\n" + - "'dynamic_partition.prefix' = 'p',\n" + - "'dynamic_partition.buckets' = '3'\n" + - " );"; + "'dynamic_partition.enable' = 'true',\n" + + "'dynamic_partition.time_unit' = 'DAY',\n" + + "'dynamic_partition.start' = '-3',\n" + + "'dynamic_partition.end' = '3',\n" + + "'dynamic_partition.prefix' = 'p',\n" + + "'dynamic_partition.buckets' = '3'\n" + + " );"; alterTableWithNewParser(stmt, false); } @@ -835,30 +836,30 @@ public void testSetDynamicPropertiesInNormalTable() throws Exception { public void testSetDynamicPropertiesInDynamicPartitionTable() throws Exception { String tableName = "dynamic_table"; String createOlapTblStmt = "CREATE TABLE test.`" + tableName + "` (\n" + - " `k1` date NULL COMMENT \"\",\n" + - " `k2` int NULL COMMENT \"\",\n" + - " `k3` smallint NULL COMMENT \"\",\n" + - " `v1` varchar(2048) NULL COMMENT \"\",\n" + - " `v2` datetime NULL COMMENT \"\"\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + - "COMMENT \"OLAP\"\n" + - "PARTITION BY RANGE (k1)\n" + - "(\n" + - "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + - "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + - "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + - ")\n" + - "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\",\n" + - "\"dynamic_partition.enable\" = \"true\",\n" + - "\"dynamic_partition.start\" = \"-3\",\n" + - "\"dynamic_partition.end\" = \"3\",\n" + - "\"dynamic_partition.time_unit\" = \"day\",\n" + - "\"dynamic_partition.prefix\" = \"p\",\n" + - "\"dynamic_partition.buckets\" = \"1\"\n" + - ");"; + " `k1` date NULL COMMENT \"\",\n" + + " `k2` int NULL COMMENT \"\",\n" + + " `k3` smallint NULL COMMENT \"\",\n" + + " `v1` varchar(2048) NULL COMMENT \"\",\n" + + " `v2` datetime NULL COMMENT \"\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + + "COMMENT \"OLAP\"\n" + + "PARTITION BY RANGE (k1)\n" + + "(\n" + + "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + + "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + + "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"dynamic_partition.enable\" = \"true\",\n" + + "\"dynamic_partition.start\" = \"-3\",\n" + + "\"dynamic_partition.end\" = \"3\",\n" + + "\"dynamic_partition.time_unit\" = \"day\",\n" + + "\"dynamic_partition.prefix\" = \"p\",\n" + + "\"dynamic_partition.buckets\" = \"1\"\n" + + ");"; createTable(createOlapTblStmt); String alterStmt1 = "alter table test." + tableName + " set (\"dynamic_partition.enable\" = \"false\");"; @@ -879,32 +880,33 @@ public void testSetDynamicPropertiesInDynamicPartitionTable() throws Exception { public void testDynamicPartitionTableMetaFailed() throws Exception { String tableName = "dynamic_table_test"; String createOlapTblStmt = "CREATE TABLE test.`" + tableName + "` (\n" + - " `k1` date NULL COMMENT \"\",\n" + - " `k2` int NULL COMMENT \"\",\n" + - " `k3` smallint NULL COMMENT \"\",\n" + - " `v1` varchar(2048) NULL COMMENT \"\",\n" + - " `v2` datetime NULL COMMENT \"\"\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + - "COMMENT \"OLAP\"\n" + - "PARTITION BY RANGE (k1)\n" + - "(\n" + - "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + - "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + - "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + - ")\n" + - "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\",\n" + - "\"dynamic_partition.enable\" = \"true\",\n" + - "\"dynamic_partition.start\" = \"-3\",\n" + - "\"dynamic_partition.end\" = \"3\",\n" + - "\"dynamic_partition.time_unit\" = \"day\",\n" + - "\"dynamic_partition.prefix\" = \"p\",\n" + - "\"dynamic_partition.buckets\" = \"1\"\n" + - ");"; + " `k1` date NULL COMMENT \"\",\n" + + " `k2` int NULL COMMENT \"\",\n" + + " `k3` smallint NULL COMMENT \"\",\n" + + " `v1` varchar(2048) NULL COMMENT \"\",\n" + + " `v2` datetime NULL COMMENT \"\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(`k1`, `k2`, `k3`)\n" + + "COMMENT \"OLAP\"\n" + + "PARTITION BY RANGE (k1)\n" + + "(\n" + + "PARTITION p1 VALUES LESS THAN (\"2014-01-01\"),\n" + + "PARTITION p2 VALUES LESS THAN (\"2014-06-01\"),\n" + + "PARTITION p3 VALUES LESS THAN (\"2014-12-01\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(`k1`) BUCKETS 32\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"dynamic_partition.enable\" = \"true\",\n" + + "\"dynamic_partition.start\" = \"-3\",\n" + + "\"dynamic_partition.end\" = \"3\",\n" + + "\"dynamic_partition.time_unit\" = \"day\",\n" + + "\"dynamic_partition.prefix\" = \"p\",\n" + + "\"dynamic_partition.buckets\" = \"1\"\n" + + ");"; createTable(createOlapTblStmt); - OlapTable olapTable = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test").getTable(tableName); + OlapTable olapTable = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getDb("test").getTable(tableName); olapTable.getTableProperty().getProperties().remove("dynamic_partition.end"); olapTable.getTableProperty().gsonPostProcess(); } @@ -912,52 +914,52 @@ public void testDynamicPartitionTableMetaFailed() throws Exception { @Test public void testSwapTable() throws Exception { String stmt1 = "CREATE TABLE test.replace1\n" + - "(\n" + - " k1 int, k2 int, k3 int sum\n" + - ")\n" + - "AGGREGATE KEY(k1, k2)\n" + - "DISTRIBUTED BY HASH(k1) BUCKETS 10\n" + - "rollup (\n" + - "r1(k1),\n" + - "r2(k2, k3)\n" + - ")\n" + - "PROPERTIES(\"replication_num\" = \"1\");"; + "(\n" + + " k1 int, k2 int, k3 int sum\n" + + ")\n" + + "AGGREGATE KEY(k1, k2)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 10\n" + + "rollup (\n" + + "r1(k1),\n" + + "r2(k2, k3)\n" + + ")\n" + + "PROPERTIES(\"replication_num\" = \"1\");"; String stmt2 = "CREATE TABLE test.r1\n" + - "(\n" + - " k1 int, k2 int\n" + - ")\n" + - "DISTRIBUTED BY HASH(k1) BUCKETS 11\n" + - "PROPERTIES(\"replication_num\" = \"1\");"; + "(\n" + + " k1 int, k2 int\n" + + ")\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 11\n" + + "PROPERTIES(\"replication_num\" = \"1\");"; String stmt3 = "CREATE TABLE test.replace2\n" + - "(\n" + - " k1 int, k2 int\n" + - ")\n" + - "DISTRIBUTED BY HASH(k1) BUCKETS 11\n" + - "PROPERTIES(\"replication_num\" = \"1\");"; + "(\n" + + " k1 int, k2 int\n" + + ")\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 11\n" + + "PROPERTIES(\"replication_num\" = \"1\");"; String stmt4 = "CREATE TABLE test.replace3\n" + - "(\n" + - " k1 int, k2 int, k3 int sum\n" + - ")\n" + - "PARTITION BY RANGE(k1)\n" + - "(\n" + - "\tPARTITION p1 values less than(\"100\"),\n" + - "\tPARTITION p2 values less than(\"200\")\n" + - ")\n" + - "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + - "rollup (\n" + - "r3(k1),\n" + - "r4(k2, k3)\n" + - ")\n" + - "PROPERTIES(\"replication_num\" = \"1\");"; + "(\n" + + " k1 int, k2 int, k3 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + "\tPARTITION p1 values less than(\"100\"),\n" + + "\tPARTITION p2 values less than(\"200\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "rollup (\n" + + "r3(k1),\n" + + "r4(k2, k3)\n" + + ")\n" + + "PROPERTIES(\"replication_num\" = \"1\");"; createTable(stmt1); createTable(stmt2); createTable(stmt3); createTable(stmt4); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); // name conflict String replaceStmt = "ALTER TABLE test.replace1 SWAP WITH r1"; @@ -966,9 +968,9 @@ public void testSwapTable() throws Exception { // replace1 with replace2 replaceStmt = "ALTER TABLE test.replace1 SWAP WITH replace2"; OlapTable replace1 = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "replace1"); + (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "replace1"); OlapTable replace2 = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "replace2"); + (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "replace2"); Assert.assertEquals(3, replace1.getPartition("replace1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE) .size()); @@ -978,8 +980,8 @@ public void testSwapTable() throws Exception { alterTableWithNewParser(replaceStmt, false); - replace1 = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "replace1"); - replace2 = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "replace2"); + replace1 = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "replace1"); + replace2 = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "replace2"); Assert.assertEquals(1, replace1.getPartition("replace1").getMaterializedIndices(MaterializedIndex.IndexExtState.VISIBLE) .size()); @@ -1009,7 +1011,7 @@ public void testSwapTableWithUniqueConstraints() throws Exception { createTable(s1); createTable(s2); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String replaceStmt = "ALTER TABLE s1 SWAP WITH s2"; alterTableWithNewParser(replaceStmt, false); @@ -1055,7 +1057,7 @@ public void testSwapTableWithForeignConstraints1() throws Exception { createTable(s1); createTable(s2); createTable(s3); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); // swap child tables String replaceStmt = "ALTER TABLE s2 SWAP WITH s3"; @@ -1130,7 +1132,7 @@ public void testSwapTableWithForeignConstraints2() throws Exception { createTable(s1); createTable(s2); createTable(s3); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); // swap parent tables String replaceStmt = "ALTER TABLE s2 SWAP WITH s1"; @@ -1178,36 +1180,36 @@ public void testCatalogAddPartitionsDay() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"2017-01-03\") END (\"2017-01-07\") EVERY (interval 1 day)"; + " PARTITIONS START (\"2017-01-03\") END (\"2017-01-07\") EVERY (interval 1 day)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNotNull(table.getPartition("p20170103")); Assert.assertNotNull(table.getPartition("p20170104")); @@ -1217,7 +1219,7 @@ public void testCatalogAddPartitionsDay() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @@ -1226,36 +1228,36 @@ public void testAddPhysicalPartition() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "DISTRIBUTED BY RANDOM BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "DISTRIBUTED BY RANDOM BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Optional partition = table.getPartitions().stream().findFirst(); Assert.assertTrue(partition.isPresent()); Assert.assertEquals(table.getPhysicalPartitions().size(), 1); - GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitions(db, table, partition.get(), 1, - WarehouseManager.DEFAULT_WAREHOUSE_ID); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addSubPartitions(db, table, partition.get(), 1, + WarehouseManager.DEFAULT_WAREHOUSE_ID); Assert.assertEquals(partition.get().getSubPartitions().size(), 2); Assert.assertEquals(table.getPhysicalPartitions().size(), 2); - GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitions(db, table, partition.get(), 2, - WarehouseManager.DEFAULT_WAREHOUSE_ID); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addSubPartitions(db, table, partition.get(), 2, + WarehouseManager.DEFAULT_WAREHOUSE_ID); Assert.assertEquals(partition.get().getSubPartitions().size(), 4); Assert.assertEquals(table.getPhysicalPartitions().size(), 4); @@ -1271,7 +1273,7 @@ public void testAddPhysicalPartition() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test @@ -1279,49 +1281,49 @@ public void testAddRangePhysicalPartition() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + - ")\n" + - "DISTRIBUTED BY RANDOM BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + + ")\n" + + "DISTRIBUTED BY RANDOM BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertEquals(table.getPhysicalPartitions().size(), 3); Partition partition = table.getPartition("p20140101"); Assert.assertNotNull(partition); - GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitions(db, table, partition, 1, - WarehouseManager.DEFAULT_WAREHOUSE_ID); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addSubPartitions(db, table, partition, 1, + WarehouseManager.DEFAULT_WAREHOUSE_ID); Assert.assertEquals(table.getPhysicalPartitions().size(), 4); Assert.assertEquals(partition.getSubPartitions().size(), 2); partition = table.getPartition("p20140103"); Assert.assertNotNull(partition); - GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitions(db, table, partition, 2, - WarehouseManager.DEFAULT_WAREHOUSE_ID); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addSubPartitions(db, table, partition, 2, + WarehouseManager.DEFAULT_WAREHOUSE_ID); Assert.assertEquals(table.getPhysicalPartitions().size(), 6); Assert.assertEquals(partition.getSubPartitions().size(), 3); dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test(expected = DdlException.class) @@ -1329,31 +1331,31 @@ public void testAddPhysicalPartitionForHash() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Optional partition = table.getPartitions().stream().findFirst(); Assert.assertTrue(partition.isPresent()); Assert.assertEquals(table.getPhysicalPartitions().size(), 1); - GlobalStateMgr.getCurrentState().getLocalMetastore().addSubPartitions(db, table, partition.get(), 1, - WarehouseManager.DEFAULT_WAREHOUSE_ID); + GlobalStateMgr.getCurrentState().getStarRocksMeta().addSubPartitions(db, table, partition.get(), 1, + WarehouseManager.DEFAULT_WAREHOUSE_ID); } @Test @@ -1383,33 +1385,33 @@ public void testAddBackend() throws Exception { public void testCatalogAddPartitions5Day() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"2017-01-03\") END (\"2017-01-15\") EVERY (interval 5 day)"; + " PARTITIONS START (\"2017-01-03\") END (\"2017-01-15\") EVERY (interval 5 day)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNotNull(table.getPartition("p20170103")); Assert.assertNotNull(table.getPartition("p20170108")); @@ -1417,38 +1419,38 @@ public void testCatalogAddPartitions5Day() throws Exception { String dropSQL = "drop table test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test(expected = AnalysisException.class) public void testCatalogAddPartitionsDayConflictException() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_exception (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20140104\") EVERY (INTERVAL 1 DAY)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition_exception ADD\n" + - " PARTITIONS START (\"2014-01-01\") END (\"2014-01-04\") EVERY (interval 1 day)"; + " PARTITIONS START (\"2014-01-01\") END (\"2014-01-04\") EVERY (interval 1 day)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exception", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exception", addPartitionClause); } @Test @@ -1456,33 +1458,33 @@ public void testCatalogAddPartitionsWeekWithoutCheck() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); Config.enable_create_partial_partition_in_batch = true; String createSQL = "CREATE TABLE test.test_partition_week (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition_week ADD\n" + - " PARTITIONS START (\"2017-03-25\") END (\"2017-04-10\") EVERY (interval 1 week)"; + " PARTITIONS START (\"2017-03-25\") END (\"2017-04-10\") EVERY (interval 1 week)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_week", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_week", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_week"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_week"); Assert.assertNotNull(table.getPartition("p2017_12")); Assert.assertNotNull(table.getPartition("p2017_13")); @@ -1490,7 +1492,7 @@ public void testCatalogAddPartitionsWeekWithoutCheck() throws Exception { String dropSQL = "drop table test_partition_week"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); Config.enable_create_partial_partition_in_batch = false; } @@ -1498,33 +1500,33 @@ public void testCatalogAddPartitionsWeekWithoutCheck() throws Exception { public void testCatalogAddPartitionsWeekWithCheck() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_week (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition_week ADD\n" + - " PARTITIONS START (\"2017-03-20\") END (\"2017-04-10\") EVERY (interval 1 week)"; + " PARTITIONS START (\"2017-03-20\") END (\"2017-04-10\") EVERY (interval 1 week)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_week", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_week", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_week"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_week"); Assert.assertNotNull(table.getPartition("p2017_12")); Assert.assertNotNull(table.getPartition("p2017_13")); @@ -1532,7 +1534,7 @@ public void testCatalogAddPartitionsWeekWithCheck() throws Exception { String dropSQL = "drop table test_partition_week"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test @@ -1540,35 +1542,35 @@ public void testCatalogAddPartitionsMonth() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"2017-01-01\") END (\"2017-04-01\") EVERY (interval 1 month)"; + " PARTITIONS START (\"2017-01-01\") END (\"2017-04-01\") EVERY (interval 1 month)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNotNull(table.getPartition("p201701")); Assert.assertNotNull(table.getPartition("p201702")); @@ -1577,7 +1579,7 @@ public void testCatalogAddPartitionsMonth() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test @@ -1585,35 +1587,35 @@ public void testCatalogAddPartitionsYear() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"2017-01-01\") END (\"2020-01-01\") EVERY (interval 1 YEAR)"; + " PARTITIONS START (\"2017-01-01\") END (\"2020-01-01\") EVERY (interval 1 YEAR)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNotNull(table.getPartition("p2017")); Assert.assertNotNull(table.getPartition("p2018")); @@ -1622,7 +1624,7 @@ public void testCatalogAddPartitionsYear() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test @@ -1630,35 +1632,35 @@ public void testCatalogAddPartitionsNumber() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test.test_partition (\n" + - " k2 INT,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 INT,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"1\") END (\"4\") EVERY (1)"; + " PARTITIONS START (\"1\") END (\"4\") EVERY (1)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNotNull(table.getPartition("p1")); Assert.assertNotNull(table.getPartition("p2")); @@ -1667,7 +1669,7 @@ public void testCatalogAddPartitionsNumber() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test @@ -1675,41 +1677,41 @@ public void testCatalogAddPartitionsAtomicRange() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); try { String alterSQL = "ALTER TABLE test_partition ADD\n" + - " PARTITIONS START (\"2014-01-01\") END (\"2014-01-06\") EVERY (interval 1 day);"; + " PARTITIONS START (\"2014-01-01\") END (\"2014-01-06\") EVERY (interval 1 day);"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); Assert.fail(); } catch (AnalysisException ex) { } - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNull(table.getPartition("p20140101")); Assert.assertNull(table.getPartition("p20140102")); @@ -1717,7 +1719,7 @@ public void testCatalogAddPartitionsAtomicRange() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @@ -1725,39 +1727,39 @@ public void testCatalogAddPartitionsZeroDay() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test_partition_0day (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140104\") END (\"20150104\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140104\") END (\"20150104\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); try { String alterSQL = "ALTER TABLE test_partition_0day ADD\n" + - " PARTITIONS START (\"2014-01-01\") END (\"2014-01-06\") EVERY (interval 0 day);"; + " PARTITIONS START (\"2014-01-01\") END (\"2014-01-06\") EVERY (interval 0 day);"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_0day", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_0day", addPartitionClause); Assert.fail(); } catch (AnalysisException ex) { } - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_0day"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_0day"); Assert.assertNull(table.getPartition("p20140101")); Assert.assertNull(table.getPartition("p20140102")); @@ -1765,7 +1767,7 @@ public void testCatalogAddPartitionsZeroDay() throws Exception { String dropSQL = "drop table test_partition_0day"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @@ -1774,41 +1776,41 @@ public void testCatalogAddPartitionsWithoutPartitions() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String dropSQL = "drop table if exists test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); String createSQL = "CREATE TABLE test_partition (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition ADD\n" + - " START (\"2015-01-01\") END (\"2015-01-06\") EVERY (interval 1 day);"; + " START (\"2015-01-01\") END (\"2015-01-06\") EVERY (interval 1 day);"; try { AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); Assert.fail(); } catch (AnalysisException ex) { } - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); Assert.assertNull(table.getPartition("p20140101")); Assert.assertNull(table.getPartition("p20140102")); @@ -1816,160 +1818,160 @@ public void testCatalogAddPartitionsWithoutPartitions() throws Exception { dropSQL = "drop table test_partition"; dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test public void testCatalogAddPartitionsIfNotExist() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test_partition_exists (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = - "ALTER TABLE test_partition_exists ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; + "ALTER TABLE test_partition_exists ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists", addPartitionClause); String alterSQL2 = - "ALTER TABLE test_partition_exists ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-02')"; + "ALTER TABLE test_partition_exists ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-02')"; AlterTableStmt alterTableStmt2 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL2, ctx); AddPartitionClause addPartitionClause2 = (AddPartitionClause) alterTableStmt2.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists", addPartitionClause2); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists", addPartitionClause2); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_exists"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_exists"); Assert.assertEquals(2, table.getPartitions().size()); String dropSQL = "drop table test_partition_exists"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test public void testCatalogAddPartitionsSameNameShouldNotThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test_partition_exists2 (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = - "ALTER TABLE test_partition_exists2 ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; + "ALTER TABLE test_partition_exists2 ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists2", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists2", addPartitionClause); String alterSQL2 = - "ALTER TABLE test_partition_exists2 ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; + "ALTER TABLE test_partition_exists2 ADD PARTITION IF NOT EXISTS p20210701 VALUES LESS THAN ('2021-07-01')"; AlterTableStmt alterTableStmt2 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL2, ctx); AddPartitionClause addPartitionClause2 = (AddPartitionClause) alterTableStmt2.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists2", addPartitionClause2); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists2", addPartitionClause2); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_exists2"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_exists2"); Assert.assertEquals(2, table.getPartitions().size()); String dropSQL = "drop table test_partition_exists2"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test(expected = AnalysisException.class) public void testCatalogAddPartitionsShouldThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test_partition_exists3 (\n" + - " k2 DATE,\n" + - " k3 SMALLINT,\n" + - " v1 VARCHAR(2048),\n" + - " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(k2, k3)\n" + - "PARTITION BY RANGE (k2) (\n" + - " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " k2 DATE,\n" + + " k3 SMALLINT,\n" + + " v1 VARCHAR(2048),\n" + + " v2 DATETIME DEFAULT \"2014-02-04 15:36:00\"\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(k2, k3)\n" + + "PARTITION BY RANGE (k2) (\n" + + " START (\"20140101\") END (\"20150101\") EVERY (INTERVAL 1 YEAR)\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); String alterSQL = "ALTER TABLE test_partition_exists3 ADD PARTITION p20210701 VALUES LESS THAN ('2021-07-01')"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL, ctx); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists3", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists3", addPartitionClause); String alterSQL2 = "ALTER TABLE test_partition_exists3 ADD PARTITION p20210701 VALUES LESS THAN ('2021-07-01')"; AlterTableStmt alterTableStmt2 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(alterSQL2, ctx); AddPartitionClause addPartitionClause2 = (AddPartitionClause) alterTableStmt2.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists3", addPartitionClause2); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_exists3", addPartitionClause2); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition_exists3"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition_exists3"); Assert.assertEquals(2, ((OlapTable) table).getPartitions().size()); String dropSQL = "drop table test_partition_exists3"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test public void testRenameDb() throws Exception { String createUserSql = "CREATE USER 'testuser' IDENTIFIED BY ''"; CreateUserStmt createUserStmt = - (CreateUserStmt) UtFrameUtils.parseStmtWithNewParser(createUserSql, starRocksAssert.getCtx()); + (CreateUserStmt) UtFrameUtils.parseStmtWithNewParser(createUserSql, starRocksAssert.getCtx()); AuthenticationMgr authenticationManager = - starRocksAssert.getCtx().getGlobalStateMgr().getAuthenticationMgr(); + starRocksAssert.getCtx().getGlobalStateMgr().getAuthenticationMgr(); authenticationManager.createUser(createUserStmt); String sql = "grant ALTER on database test to testuser"; DDLStmtExecutor.execute(UtFrameUtils.parseStmtWithNewParser(sql, starRocksAssert.getCtx()), - starRocksAssert.getCtx()); + starRocksAssert.getCtx()); UserIdentity testUser = new UserIdentity("testuser", "%"); testUser.analyze(); @@ -1977,13 +1979,13 @@ public void testRenameDb() throws Exception { starRocksAssert.getCtx().setQualifiedUser("testuser"); starRocksAssert.getCtx().setCurrentUserIdentity(testUser); starRocksAssert.getCtx().setCurrentRoleIds( - GlobalStateMgr.getCurrentState().getAuthorizationMgr().getRoleIdsByUser(testUser)); + GlobalStateMgr.getCurrentState().getAuthorizationMgr().getRoleIdsByUser(testUser)); starRocksAssert.getCtx().setRemoteIP("%"); starRocksAssert.withDatabase("test_to_rename"); String renameDb = "alter database test_to_rename rename test_to_rename_2"; AlterDatabaseRenameStatement renameDbStmt = - (AlterDatabaseRenameStatement) UtFrameUtils.parseStmtWithNewParser(renameDb, starRocksAssert.getCtx()); + (AlterDatabaseRenameStatement) UtFrameUtils.parseStmtWithNewParser(renameDb, starRocksAssert.getCtx()); DDLStmtExecutor.execute(renameDbStmt, starRocksAssert.getCtx()); } @@ -1991,38 +1993,38 @@ public void testRenameDb() throws Exception { public void testAddMultiItemListPartition() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10) not null,\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (dt,province) (\n" + - " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + - " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10) not null,\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (dt,province) (\n" + + " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + + " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values = Lists.newArrayList("2022-04-01", "shandong"); List> multiValues = Lists.newArrayList(); multiValues.add(values); PartitionDesc partitionDesc = new MultiItemListPartitionDesc(false, "p3", multiValues, new HashMap<>()); AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); - OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test") - .getTable("test_partition"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test") + .getTable("test_partition"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); ListPartitionInfo partitionInfo = (ListPartitionInfo) table.getPartitionInfo(); Map>> idToValues = partitionInfo.getIdToMultiValues(); @@ -2034,62 +2036,63 @@ public void testAddMultiItemListPartition() throws Exception { String dropSQL = "drop table test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test(expected = AlterJobException.class) public void testModifyPartitionBucket() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE modify_bucket (\n" + - " chuangyi varchar(65533) NULL COMMENT \"创意\",\n" + - " guanggao varchar(65533) NULL COMMENT \"广告\"\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(chuangyi, guanggao)\n" + - "COMMENT \"OLAP\"\n" + - "DISTRIBUTED BY HASH(chuangyi, guanggao) BUCKETS 3\n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\"\n" + - ");"; + " chuangyi varchar(65533) NULL COMMENT \"创意\",\n" + + " guanggao varchar(65533) NULL COMMENT \"广告\"\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(chuangyi, guanggao)\n" + + "COMMENT \"OLAP\"\n" + + "DISTRIBUTED BY HASH(chuangyi, guanggao) BUCKETS 3\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\"\n" + + ");"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); String stmt = "alter table modify_bucket set (\"dynamic_partition.buckets\" = \"10\");\n"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(stmt, starRocksAssert.getCtx()); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); } @Test public void testAddSingleItemListPartition() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10),\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (province) (\n" + - " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + - " PARTITION p2 VALUES IN (\"guangdong\") \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + + " PARTITION p2 VALUES IN (\"guangdong\") \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values = Lists.newArrayList("shanxi", "shanghai"); PartitionDesc partitionDesc = new SingleItemListPartitionDesc(false, "p3", values, new HashMap<>()); - AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); + AddPartitionClause addPartitionClause = + new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); OlapTable table = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition"); + (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "test_partition"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition", addPartitionClause); ListPartitionInfo partitionInfo = (ListPartitionInfo) table.getPartitionInfo(); Map> idToValues = partitionInfo.getIdToValues(); @@ -2100,33 +2103,33 @@ public void testAddSingleItemListPartition() throws Exception { String dropSQL = "drop table test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); } @Test public void testSingleItemPartitionPersistInfo() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10),\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (province) (\n" + - " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable table = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getTable(db.getFullName(), "test_partition"); ListPartitionInfo partitionInfo = (ListPartitionInfo) table.getPartitionInfo(); long dbId = db.getId(); @@ -2139,8 +2142,8 @@ public void testSingleItemPartitionPersistInfo() throws Exception { boolean isInMemory = partitionInfo.getIsInMemory(partitionId); boolean isTempPartition = false; ListPartitionPersistInfo partitionPersistInfoOut = new ListPartitionPersistInfo(dbId, tableId, partition, - dataProperty, replicationNum, isInMemory, isTempPartition, values, new ArrayList<>(), - partitionInfo.getDataCacheInfo(partitionId)); + dataProperty, replicationNum, isInMemory, isTempPartition, values, new ArrayList<>(), + partitionInfo.getDataCacheInfo(partitionId)); // write log File file = new File("./test_serial.log"); @@ -2177,7 +2180,7 @@ public void testSingleItemPartitionPersistInfo() throws Exception { String dropSQL = "drop table test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); file.delete(); } @@ -2185,26 +2188,26 @@ public void testSingleItemPartitionPersistInfo() throws Exception { public void testMultiItemPartitionPersistInfo() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10) not null,\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (dt , province) (\n" + - " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\"))\n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10) not null,\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (dt , province) (\n" + + " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\"))\n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable table = - (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta() + .getTable(db.getFullName(), "test_partition"); ListPartitionInfo partitionInfo = (ListPartitionInfo) table.getPartitionInfo(); long dbId = db.getId(); @@ -2217,8 +2220,8 @@ public void testMultiItemPartitionPersistInfo() throws Exception { boolean isInMemory = partitionInfo.getIsInMemory(partitionId); boolean isTempPartition = false; ListPartitionPersistInfo partitionPersistInfoOut = new ListPartitionPersistInfo(dbId, tableId, partition, - dataProperty, replicationNum, isInMemory, isTempPartition, new ArrayList<>(), multiValues, - partitionInfo.getDataCacheInfo(partitionId)); + dataProperty, replicationNum, isInMemory, isTempPartition, new ArrayList<>(), multiValues, + partitionInfo.getDataCacheInfo(partitionId)); // write log File file = new File("./test_serial.log"); @@ -2259,7 +2262,7 @@ public void testMultiItemPartitionPersistInfo() throws Exception { String dropSQL = "drop table test_partition"; DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser(dropSQL, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); file.delete(); } @@ -2267,61 +2270,61 @@ public void testMultiItemPartitionPersistInfo() throws Exception { public void testAddSingleListPartitionSamePartitionNameShouldThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_1 (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10),\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (province) (\n" + - " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + - " PARTITION p2 VALUES IN (\"guangdong\") \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + + " PARTITION p2 VALUES IN (\"guangdong\") \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values = Lists.newArrayList("shanxi", "heilongjiang"); PartitionDesc partitionDesc = new SingleItemListPartitionDesc(false, "p1", values, new HashMap<>()); AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition_1"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "test_partition_1"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_1", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_1", addPartitionClause); } @Test(expected = SemanticException.class) public void testAddMultiListPartitionSamePartitionNameShouldThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_2 (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10) not null,\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (dt,province) (\n" + - " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + - " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10) not null,\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (dt,province) (\n" + + " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + + " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values1 = Lists.newArrayList("2022-04-01", "beijing"); List values2 = Lists.newArrayList("2022-04-01", "chongqing"); @@ -2331,41 +2334,41 @@ public void testAddMultiListPartitionSamePartitionNameShouldThrowError() throws PartitionDesc partitionDesc = new MultiItemListPartitionDesc(false, "p1", multiValues, new HashMap<>()); AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition_2"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "test_partition_2"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_2", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "test_partition_2", addPartitionClause); } @Test(expected = SemanticException.class) public void testAddSingleListPartitionSamePartitionValueShouldThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_3 (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10),\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (province) (\n" + - " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + - " PARTITION p2 VALUES IN (\"guangdong\") \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN (\"beijing\",\"chongqing\") ,\n" + + " PARTITION p2 VALUES IN (\"guangdong\") \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values = Lists.newArrayList("beijing", "chongqing"); PartitionDesc partitionDesc = new SingleItemListPartitionDesc(false, "p3", values, new HashMap<>()); AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition_3"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "test_partition_3"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); } @@ -2374,25 +2377,25 @@ public void testAddSingleListPartitionSamePartitionValueShouldThrowError() throw public void testAddMultiItemListPartitionSamePartitionValueShouldThrowError() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String createSQL = "CREATE TABLE test.test_partition_4 (\n" + - " id BIGINT,\n" + - " age SMALLINT,\n" + - " dt VARCHAR(10) not null,\n" + - " province VARCHAR(64) not null\n" + - ")\n" + - "ENGINE=olap\n" + - "DUPLICATE KEY(id)\n" + - "PARTITION BY LIST (dt, province) (\n" + - " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + - " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + - ")\n" + - "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + - "PROPERTIES (\n" + - " \"replication_num\" = \"1\"\n" + - ")"; + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10) not null,\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (dt, province) (\n" + + " PARTITION p1 VALUES IN ((\"2022-04-01\", \"beijing\"),(\"2022-04-01\", \"chongqing\")),\n" + + " PARTITION p2 VALUES IN ((\"2022-04-01\", \"shanghai\")) \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); List values = Lists.newArrayList("2022-04-01", "shanghai"); List> multiValues = Lists.newArrayList(); @@ -2400,7 +2403,7 @@ public void testAddMultiItemListPartitionSamePartitionValueShouldThrowError() th PartitionDesc partitionDesc = new MultiItemListPartitionDesc(false, "p3", multiValues, new HashMap<>()); AddPartitionClause addPartitionClause = new AddPartitionClause(partitionDesc, null, new HashMap<>(), false); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "test_partition_4"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "test_partition_4"); AlterTableClauseAnalyzer analyzer = new AlterTableClauseAnalyzer(table); analyzer.analyze(Util.getOrCreateConnectContext(), addPartitionClause); } @@ -2408,16 +2411,16 @@ public void testAddMultiItemListPartitionSamePartitionValueShouldThrowError() th @Test public void testCatalogAddColumn() throws Exception { starRocksAssert.withDatabase("test").useDatabase("test") - .withTable("CREATE TABLE test.tbl1\n" + - "(\n" + - " k1 date,\n" + - " v1 int \n" + - ")\n" + - "DUPLICATE KEY(`k1`)" + - "DISTRIBUTED BY HASH (k1) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); - OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "tbl1"); + .withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 date,\n" + + " v1 int \n" + + ")\n" + + "DUPLICATE KEY(`k1`)" + + "DISTRIBUTED BY HASH (k1) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test"); + OlapTable tbl = (OlapTable) GlobalStateMgr.getCurrentState().getStarRocksMeta().getTable(db.getFullName(), "tbl1"); String stmt = "alter table test.tbl1 add column k2 int"; alterTableWithNewParser(stmt, false); @@ -2439,7 +2442,7 @@ public void testCatalogAddColumn() throws Exception { @Test public void testCatalogAddColumns() throws Exception { String stmt = "alter table test.tbl1 add column (`col1` int(11) not null default \"0\" comment \"\", " - + "`col2` int(11) not null default \"0\" comment \"\") in `testTable`;"; + + "`col2` int(11) not null default \"0\" comment \"\") in `testTable`;"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(stmt, starRocksAssert.getCtx()); AddColumnsClause clause = (AddColumnsClause) alterTableStmt.getAlterClauseList().get(0); Assert.assertEquals(2, clause.getColumns().size()); @@ -2447,7 +2450,7 @@ public void testCatalogAddColumns() throws Exception { Assert.assertEquals("testTable", clause.getRollupName()); stmt = "alter table test.tbl1 add column (`col1` int(11) not null default \"0\" comment \"\", " - + "`col2` int(11) not null default \"0\" comment \"\");"; + + "`col2` int(11) not null default \"0\" comment \"\");"; alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(stmt, starRocksAssert.getCtx()); clause = (AddColumnsClause) alterTableStmt.getAlterClauseList().get(0); Assert.assertEquals(null, clause.getRollupName()); @@ -2458,35 +2461,35 @@ public void testCreateTemporaryPartitionInBatch() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); starRocksAssert.withDatabase("test2"); String createSQL = "CREATE TABLE test2.site_access(\n" + - " event_day datetime,\n" + - " site_id INT DEFAULT '10',\n" + - " city_code VARCHAR(100),\n" + - " user_name VARCHAR(32) DEFAULT '',\n" + - " pv BIGINT DEFAULT '0'\n" + - ")\n" + - "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + - "PARTITION BY date_trunc('day', event_day)(\n" + - " START (\"2023-03-27\") END (\"2023-03-30\") EVERY (INTERVAL 1 day)\n" + - ")\n" + - "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 32\n" + - "PROPERTIES(\n" + - " \"replication_num\" = \"1\"\n" + - ");"; + " event_day datetime,\n" + + " site_id INT DEFAULT '10',\n" + + " city_code VARCHAR(100),\n" + + " user_name VARCHAR(32) DEFAULT '',\n" + + " pv BIGINT DEFAULT '0'\n" + + ")\n" + + "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + + "PARTITION BY date_trunc('day', event_day)(\n" + + " START (\"2023-03-27\") END (\"2023-03-30\") EVERY (INTERVAL 1 day)\n" + + ")\n" + + "DISTRIBUTED BY HASH(event_day, site_id) BUCKETS 32\n" + + "PROPERTIES(\n" + + " \"replication_num\" = \"1\"\n" + + ");"; CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser(createSQL, ctx); StarRocksAssert.utCreateTableWithRetry(createTableStmt); - Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test2"); + Database db = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test2"); String sql = "alter table test2.site_access add TEMPORARY partitions " + - "START (\"2023-03-27\") END (\"2023-03-30\") EVERY (INTERVAL 1 day);"; + "START (\"2023-03-27\") END (\"2023-03-30\") EVERY (INTERVAL 1 day);"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, starRocksAssert.getCtx()); AddPartitionClause addPartitionClause = (AddPartitionClause) alterTableStmt.getAlterClauseList().get(0); - GlobalStateMgr.getCurrentState().getLocalMetastore() - .addPartitions(Util.getOrCreateConnectContext(), db, "site_access", addPartitionClause); + GlobalStateMgr.getCurrentState().getStarRocksMeta() + .addPartitions(Util.getOrCreateConnectContext(), db, "site_access", addPartitionClause); - Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test2") - .getTable("site_access"); + Table table = GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test2") + .getTable("site_access"); OlapTable olapTable = (OlapTable) table; PartitionInfo partitionInfo = olapTable.getPartitionInfo(); RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; @@ -2497,14 +2500,14 @@ public void testCreateTemporaryPartitionInBatch() throws Exception { @Test public void testCatalogDropColumn() throws Exception { starRocksAssert.withDatabase("test").useDatabase("test") - .withTable("CREATE TABLE test.tbl1\n" + - "(\n" + - " k1 date,\n" + - " k2 int,\n" + - " v1 int sum\n" + - ")\n" + - "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + - "PROPERTIES('replication_num' = '1');"); + .withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + + "PROPERTIES('replication_num' = '1');"); String stmt = "alter table test.tbl1 drop column k2 from `testRollup`"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(stmt, starRocksAssert.getCtx()); DropColumnClause clause = (DropColumnClause) alterTableStmt.getAlterClauseList().get(0); @@ -2605,9 +2608,9 @@ public Table getTable(String tableName) { TruncatePartitionClause clause = new TruncatePartitionClause(partitionNames); cList.add(clause); AlterJobMgr alter = new AlterJobMgr( - new SchemaChangeHandler(), - new MaterializedViewHandler(), - new SystemHandler()); + new SchemaChangeHandler(), + new MaterializedViewHandler(), + new SystemHandler()); TableName tableName = new TableName("test_db", "test_table"); AlterTableStmt stmt = new AlterTableStmt(tableName, cList); DDLStmtExecutor.execute(stmt, starRocksAssert.getCtx()); @@ -2626,69 +2629,69 @@ public void testAutoPartitionTableUnsupported() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String sql = "ALTER TABLE site_access_date_trunc ADD PARTITION p20210101 VALUES [(\"2021-01-01\"), (\"2021-01-02\"));"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); } @Test(expected = AnalysisException.class) public void testAutoPartitionTableUnsupported2() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String sql = "ALTER TABLE site_access_time_slice\n" + - "ADD PARTITIONS START (\"2022-05-01\") END (\"2022-05-03\") EVERY (INTERVAL 1 day)"; + "ADD PARTITIONS START (\"2022-05-01\") END (\"2022-05-03\") EVERY (INTERVAL 1 day)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); } @Test(expected = AnalysisException.class) public void testAutoPartitionTableUnsupported3() throws Exception { ConnectContext ctx = starRocksAssert.getCtx(); String sql = "ALTER TABLE site_access_date_trunc\n" + - "ADD PARTITIONS START (\"2022-05-01\") END (\"2022-05-03\") EVERY (INTERVAL 2 day)"; + "ADD PARTITIONS START (\"2022-05-01\") END (\"2022-05-03\") EVERY (INTERVAL 2 day)"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); } @Test public void testAlterMvWithResourceGroup() throws Exception { starRocksAssert.executeResourceGroupDdlSql("create resource group if not exists mv_rg" + - " with (" + - " 'cpu_core_limit' = '10'," + - " 'mem_limit' = '20%'," + - " 'concurrency_limit' = '11'," + - " 'type' = 'mv'" + - " );"); + " with (" + + " 'cpu_core_limit' = '10'," + + " 'mem_limit' = '20%'," + + " 'concurrency_limit' = '11'," + + " 'type' = 'mv'" + + " );"); starRocksAssert.useDatabase("test") - .withMaterializedView("CREATE MATERIALIZED VIEW `mv2` (a comment \"a1\", b comment \"b2\", c)\n" + - "COMMENT \"MATERIALIZED_VIEW\"\n" + - "DISTRIBUTED BY HASH(a) BUCKETS 12\n" + - "REFRESH ASYNC\n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\",\n" + - "\"replicated_storage\" = \"true\",\n" + - "\"resource_group\" = \"mv_rg\",\n" + - "\"storage_medium\" = \"HDD\"\n" + - ")\n" + - "AS SELECT k1, k2, v1 from test.tbl1"); + .withMaterializedView("CREATE MATERIALIZED VIEW `mv2` (a comment \"a1\", b comment \"b2\", c)\n" + + "COMMENT \"MATERIALIZED_VIEW\"\n" + + "DISTRIBUTED BY HASH(a) BUCKETS 12\n" + + "REFRESH ASYNC\n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"replicated_storage\" = \"true\",\n" + + "\"resource_group\" = \"mv_rg\",\n" + + "\"storage_medium\" = \"HDD\"\n" + + ")\n" + + "AS SELECT k1, k2, v1 from test.tbl1"); MaterializedView mv = - (MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test").getTable("mv2"); + (MaterializedView) GlobalStateMgr.getCurrentState().getStarRocksMeta().getDb("test").getTable("mv2"); Assert.assertEquals("mv_rg", mv.getTableProperty().getResourceGroup()); ConnectContext ctx = starRocksAssert.getCtx(); String sql = "ALTER MATERIALIZED VIEW mv2\n" + - "set (\"resource_group\" =\"\" )"; + "set (\"resource_group\" =\"\" )"; AlterMaterializedViewStmt alterTableStmt = (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterTableStmt); Assert.assertEquals("", mv.getTableProperty().getResourceGroup()); sql = "ALTER MATERIALIZED VIEW mv2\n" + - "set (\"resource_group\" =\"not_exist_rg\" )"; + "set (\"resource_group\" =\"not_exist_rg\" )"; AlterMaterializedViewStmt alterTableStmt2 = - (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); Assert.assertThrows("resource_group not_exist_rg does not exist.", - SemanticException.class, - () -> GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterTableStmt2)); + SemanticException.class, + () -> GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterTableStmt2)); sql = "ALTER MATERIALIZED VIEW mv2\n" + - "set (\"resource_group\" =\"mv_rg\" )"; + "set (\"resource_group\" =\"mv_rg\" )"; AlterMaterializedViewStmt alterTableStmt3 = - (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterTableStmt3); + (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterTableStmt3); Assert.assertEquals("mv_rg", mv.getTableProperty().getResourceGroup()); new MockUp() { @@ -2699,31 +2702,31 @@ public Warehouse getWarehouse(String warehouseName) { }; sql = "ALTER MATERIALIZED VIEW mv2 set (\"warehouse\" = \"w1\")"; AlterMaterializedViewStmt alterTableStmt4 = - (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterTableStmt4); + (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterTableStmt4); Assert.assertEquals(1L, mv.getWarehouseId()); } @Test(expected = ErrorReportException.class) public void testAlterListPartitionUseBatchBuildPartition() throws Exception { starRocksAssert.useDatabase("test").withTable("CREATE TABLE t2 (\n" + - " dt datetime not null,\n" + - " user_id bigint not null,\n" + - " recharge_money decimal(32,2) not null, \n" + - " province varchar(20) not null,\n" + - " id varchar(20) not null\n" + - ") ENGINE=OLAP\n" + - "DUPLICATE KEY(dt)\n" + - "PARTITION BY (dt)\n" + - "DISTRIBUTED BY HASH(`dt`) BUCKETS 10 \n" + - "PROPERTIES (\n" + - "\"replication_num\" = \"1\",\n" + - "\"in_memory\" = \"false\"\n" + - ");"); + " dt datetime not null,\n" + + " user_id bigint not null,\n" + + " recharge_money decimal(32,2) not null, \n" + + " province varchar(20) not null,\n" + + " id varchar(20) not null\n" + + ") ENGINE=OLAP\n" + + "DUPLICATE KEY(dt)\n" + + "PARTITION BY (dt)\n" + + "DISTRIBUTED BY HASH(`dt`) BUCKETS 10 \n" + + "PROPERTIES (\n" + + "\"replication_num\" = \"1\",\n" + + "\"in_memory\" = \"false\"\n" + + ");"); ConnectContext ctx = starRocksAssert.getCtx(); String sql = "ALTER TABLE t2 ADD PARTITIONS START (\"2021-01-04\") END (\"2021-01-06\") EVERY (INTERVAL 1 DAY);"; AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); } @Test @@ -2733,25 +2736,25 @@ public void testAlterForeignKey() throws Exception { { // inner table starRocksAssert.useDatabase("test").withMaterializedView("create materialized view if not exists `fk_mv_1` " + - "refresh manual " + - "as " + - "select t1.event_day, t1.site_id, t2.user_name " + - "from site_access_date_trunc t1 join site_access_time_slice t2 " + - "on t1.site_id = t2.site_id"); + "refresh manual " + + "as " + + "select t1.event_day, t1.site_id, t2.user_name " + + "from site_access_date_trunc t1 join site_access_time_slice t2 " + + "on t1.site_id = t2.site_id"); connectContext.executeSql("alter materialized view fk_mv_1 set " + - "( 'unique_constraints'='site_access_date_trunc.site_id'); "); + "( 'unique_constraints'='site_access_date_trunc.site_id'); "); connectContext.executeSql("alter materialized view fk_mv_1 set " + - "( 'foreign_key_constraints'='site_access_time_slice(site_id)" + - " REFERENCES site_access_date_trunc(site_id)'); "); + "( 'foreign_key_constraints'='site_access_time_slice(site_id)" + + " REFERENCES site_access_date_trunc(site_id)'); "); while (true) { ModifyTablePropertyOperationLog modifyMvLog = - (ModifyTablePropertyOperationLog) UtFrameUtils.PseudoJournalReplayer. - replayNextJournal(OperationType.OP_ALTER_MATERIALIZED_VIEW_PROPERTIES); + (ModifyTablePropertyOperationLog) UtFrameUtils.PseudoJournalReplayer. + replayNextJournal(OperationType.OP_ALTER_MATERIALIZED_VIEW_PROPERTIES); Assert.assertNotNull(modifyMvLog); if (modifyMvLog.getProperties().containsKey("foreign_key_constraints")) { Assert.assertEquals("default_catalog.10001.10133(site_id) " + - "REFERENCES default_catalog.10001.10118(site_id)", - modifyMvLog.getProperties().get("foreign_key_constraints")); + "REFERENCES default_catalog.10001.10118(site_id)", + modifyMvLog.getProperties().get("foreign_key_constraints")); break; } } @@ -2760,24 +2763,24 @@ public void testAlterForeignKey() throws Exception { { // external table starRocksAssert.withMaterializedView("create materialized view if not exists `fk_mv_2` " + - "refresh manual " + - "as " + - "select t1.l_orderkey, t1.l_partkey, t2.o_totalprice " + - "from hive0.tpch.lineitem t1 join hive0.tpch.orders t2 " + - "on t1.l_orderkey = t2.o_orderkey"); + "refresh manual " + + "as " + + "select t1.l_orderkey, t1.l_partkey, t2.o_totalprice " + + "from hive0.tpch.lineitem t1 join hive0.tpch.orders t2 " + + "on t1.l_orderkey = t2.o_orderkey"); connectContext.executeSql("alter materialized view fk_mv_2 set " + - "( 'unique_constraints'='hive0.tpch.orders.o_orderkey'); "); + "( 'unique_constraints'='hive0.tpch.orders.o_orderkey'); "); connectContext.executeSql("alter materialized view fk_mv_2 set " + - "( 'foreign_key_constraints'='hive0.tpch.lineitem(l_orderkey) " + - "REFERENCES hive0.tpch.orders(o_orderkey)'); "); + "( 'foreign_key_constraints'='hive0.tpch.lineitem(l_orderkey) " + + "REFERENCES hive0.tpch.orders(o_orderkey)'); "); while (true) { ModifyTablePropertyOperationLog modifyMvLog = - (ModifyTablePropertyOperationLog) UtFrameUtils.PseudoJournalReplayer. - replayNextJournal(OperationType.OP_ALTER_MATERIALIZED_VIEW_PROPERTIES); + (ModifyTablePropertyOperationLog) UtFrameUtils.PseudoJournalReplayer. + replayNextJournal(OperationType.OP_ALTER_MATERIALIZED_VIEW_PROPERTIES); Assert.assertNotNull(modifyMvLog); if (modifyMvLog.getProperties().containsKey("foreign_key_constraints")) { Assert.assertEquals("hive0.tpch.lineitem:0(l_orderkey) REFERENCES hive0.tpch.orders:0(o_orderkey)", - modifyMvLog.getProperties().get("foreign_key_constraints")); + modifyMvLog.getProperties().get("foreign_key_constraints")); break; } } diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java index b990fdace4b53..5d62e4864dc31 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java @@ -619,7 +619,7 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception { "as select date_trunc('month',tb1.k1) s1, tb2.k2 s2 from tbl1 tb1 join tbl2 tb2 on tb1.k2 = tb2.k2;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); Table mv1 = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "mv1"); Assert.assertTrue(mv1 instanceof MaterializedView); // test partition @@ -686,7 +686,7 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception { "as select k1, tbl1.k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); Table mv1 = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "mv1"); Assert.assertTrue(mv1 instanceof MaterializedView); // test partition @@ -742,7 +742,7 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception { "as select k1, tbl1.k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.fail(e.getMessage()); } finally { @@ -1626,7 +1626,7 @@ public void testCreateMvFromMv() { "as select k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql1, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -1640,7 +1640,7 @@ public void testCreateMvFromMv() { "as select k1, k2 from base_mv;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql2, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -1658,7 +1658,7 @@ public void testCreateMvFromMv2() throws Exception { "as select k1, k2 from tbl1;"; { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql1, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } String sql2 = "create materialized view mv_from_base_mv2 " + @@ -1671,7 +1671,7 @@ public void testCreateMvFromMv2() throws Exception { "as select k1, k2 from base_mv2;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql2, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -1689,7 +1689,7 @@ public void testCreateMvFromInactiveMv() { "as select k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql1, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -1707,7 +1707,7 @@ public void testCreateMvFromInactiveMv() { "as select k1, k2 from base_inactive_mv;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql2, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.assertEquals("Getting analyzing error at line 3, column 24. Detail message: " + "Create/Rebuild materialized view from inactive materialized view: base_inactive_mv.", @@ -1727,7 +1727,7 @@ public void testAsHasStar() throws Exception { "as select k1 ss, * from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); MaterializedView mv = ((MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "testAsHasStar")); mv.setInactiveAndReason(""); List mvColumns = mv.getFullSchema(); @@ -1760,7 +1760,7 @@ public void testAsHasStarWithSameColumns() throws Exception { "as select a.k1 ss, a.*, b.* from tbl1 as a join tbl1 as b on a.k1=b.k1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); Assert.fail(); } catch (Exception e) { Assert.assertTrue(e.getMessage().contains("Duplicate column name 'k1'")); @@ -1781,7 +1781,7 @@ public void testMVWithSameColumns() throws Exception { "as select a.k1 ss, a.k2, b.k2 from tbl1 as a join tbl1 as b on a.k1=b.k1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); Assert.fail(); } catch (Exception e) { Assert.assertTrue(e.getMessage().contains("Duplicate column name 'k2'")); @@ -1820,7 +1820,7 @@ public void testAsSelectItemAlias1() throws Exception { "as select date_trunc('month',tbl1.k1), k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); MaterializedView mv = ((MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "testAsSelectItemAlias1")); mv.setInactiveAndReason(""); List mvColumns = mv.getFullSchema(); @@ -1851,7 +1851,7 @@ public void testAsSelectItemAlias2() throws Exception { "select date_trunc('month',tbl1.k1), k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); MaterializedView mv = ((MaterializedView) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "testAsSelectItemAlias2")); mv.setInactiveAndReason(""); List mvColumns = mv.getFullSchema(); @@ -1878,7 +1878,7 @@ public void testAsSelectItemAlias3() { "as select date_trunc('month',tbl1.k1), k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.assertTrue(e.getMessage().contains("Materialized view partition exp: " + "`tbl1`.`k1` must related to column")); @@ -1898,7 +1898,7 @@ public void testAsSelectItemAlias4() { "as select date_trunc('month',tbl1.k1), k1, k2 from tbl1;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); } catch (Exception e) { Assert.assertTrue(e.getMessage() .contains("No viable statement for input 'distributed by hash(date_trunc('.")); @@ -2001,7 +2001,7 @@ public boolean isEnableColocateMVIndex() throws Exception { "as select k1, k2 from colocateTable;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStmt) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStmt) statementBase); waitingRollupJobV2Finish(); ColocateTableIndex colocateTableIndex = currentState.getColocateTableIndex(); String fullGroupName = testDb.getId() + "_" + "colocate_group1"; @@ -2048,7 +2048,7 @@ public void testCreateColocateMvWithoutGroup() throws Exception { Assert.assertThrows(AnalysisException.class, () -> { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStmt) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStmt) statementBase); }); currentState.getColocateTableIndex().clear(); @@ -2097,10 +2097,10 @@ public boolean isEnableColocateMVIndex() throws Exception { "as select k1, k2 from colocateTable3;"; try { StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStmt) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStmt) statementBase); waitingRollupJobV2Finish(); statementBase = UtFrameUtils.parseStmtWithNewParser(sql2, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStmt) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStmt) statementBase); waitingRollupJobV2Finish(); ColocateTableIndex colocateTableIndex = currentState.getColocateTableIndex(); @@ -2251,7 +2251,7 @@ private void assertCreateFailWithException(String sql, String msg) { } try { - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); Assert.fail(); } catch (Exception e) { Assert.assertTrue(e.getMessage().contains(msg)); @@ -2287,7 +2287,7 @@ public void testCreateMVWithSessionProperties1() { CreateMaterializedViewStatement stmt = (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, starRocksAssert.getCtx()); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); Table mv1 = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "mv_with_property1"); Assert.assertTrue(mv1 instanceof MaterializedView); } catch (Exception e) { @@ -2337,7 +2337,7 @@ public void testNoDuplicateKey() { CreateMaterializedViewStatement stmt = (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); } catch (Exception e) { Assert.fail(); } @@ -2470,7 +2470,7 @@ public void testCreateMvWithColocateGroup() throws Exception { ") " + "as select tbl1.k1 ss, k2 from tbl1;"; StatementBase statementBase = UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView((CreateMaterializedViewStatement) statementBase); + currentState.getStarRocksMeta().createMaterializedView((CreateMaterializedViewStatement) statementBase); String fullGroupName = testDb.getId() + "_" + groupName; long tableId = currentState.getColocateTableIndex().getTableIdByGroup(fullGroupName); Assert.assertTrue(tableId > 0); @@ -2544,7 +2544,7 @@ public void testMvNameInvalid() { CreateMaterializedViewStatement stmt = (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); } catch (Exception e) { Assert.fail(); } @@ -2564,7 +2564,7 @@ public void testMvName1() { CreateMaterializedViewStatement stmt = (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); } catch (Exception e) { Assert.fail(); } @@ -2584,7 +2584,7 @@ public void testPartitionAndDistributionByColumnNameIgnoreCase() { CreateMaterializedViewStatement stmt = (CreateMaterializedViewStatement) UtFrameUtils.parseStmtWithNewParser(sql, connectContext); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -2954,7 +2954,7 @@ public void testCreateMVWithDifferentDB() { (CreateMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, newStarRocksAssert.getCtx()); Assert.assertEquals(stmt.getDBName(), "test"); Assert.assertEquals(stmt.getMVName(), "test_mv_use_different_tbl"); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); waitingRollupJobV2Finish(); Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "tbl5"); @@ -3008,7 +3008,7 @@ public void testCreateAsyncMVWithDifferentDB() { Assert.assertEquals(stmt.getTableName().getDb(), "test"); Assert.assertEquals(stmt.getTableName().getTbl(), "test_mv_use_different_tbl"); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); newStarRocksAssert.dropDatabase("test_mv_different_db"); Table mv1 = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(testDb.getFullName(), "test_mv_use_different_tbl"); Assert.assertTrue(mv1 instanceof MaterializedView); @@ -3034,7 +3034,7 @@ public void testCreateAsyncMVWithDifferentDB2() { Assert.assertEquals(stmt.getTableName().getDb(), "test_mv_different_db"); Assert.assertEquals(stmt.getTableName().getTbl(), "test_mv_use_different_tbl"); - currentState.getLocalMetastore().createMaterializedView(stmt); + currentState.getStarRocksMeta().createMaterializedView(stmt); Database differentDb = currentState.getLocalMetastore().getDb("test_mv_different_db"); Table mv1 = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(differentDb.getFullName(), "test_mv_use_different_tbl"); @@ -3880,7 +3880,7 @@ MaterializedView getMaterializedViewChecked(String sql) { CreateMaterializedViewStatement createMaterializedViewStatement = (CreateMaterializedViewStatement) statementBase; - currentState.getLocalMetastore().createMaterializedView(createMaterializedViewStatement); + currentState.getStarRocksMeta().createMaterializedView(createMaterializedViewStatement); ThreadUtil.sleepAtLeastIgnoreInterrupts(4000L); TableName mvName = createMaterializedViewStatement.getTableName(); @@ -3902,7 +3902,7 @@ List getMaterializedViewKeysChecked(String sql) { CreateMaterializedViewStatement createMaterializedViewStatement = (CreateMaterializedViewStatement) statementBase; - currentState.getLocalMetastore().createMaterializedView(createMaterializedViewStatement); + currentState.getStarRocksMeta().createMaterializedView(createMaterializedViewStatement); ThreadUtil.sleepAtLeastIgnoreInterrupts(4000L); TableName mvTableName = createMaterializedViewStatement.getTableName(); diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java index a8e9a75593a5b..91c8f733f26a1 100644 --- a/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java @@ -200,7 +200,7 @@ public void testMaxMVRewriteStaleness1() { "set (\"mv_rewrite_staleness_second\" = \"%s\")", MV_STALENESS); AlterMaterializedViewStmt stmt = (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(alterMvSql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(stmt); } // no refresh partitions if mv_rewrite_staleness is set. executeInsertSql(connectContext, "insert into tbl_staleness1 values(\"2022-02-20\", 1, 10)"); @@ -431,7 +431,7 @@ public void testMaxMVRewriteStaleness3() { AlterMaterializedViewStmt stmt = (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(alterMvSql, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(stmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(stmt); } MaterializedView mv1 = getMv("test", "mv_with_mv_rewrite_staleness21"); diff --git a/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobMaterializedViewTest.java index b5dde67a513e8..bc104cc1a3fb5 100644 --- a/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobMaterializedViewTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/backup/RestoreJobMaterializedViewTest.java @@ -203,7 +203,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { //minTimes = 0; //result = systemInfoService; - globalStateMgr.getLocalMetastore().mayGetDb(anyLong); + globalStateMgr.getStarRocksMeta().mayGetDb(anyLong); minTimes = 0; result = Optional.of(db); diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/AdminStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/AdminStmtTest.java index 06db0098cd87c..949417ccd4bc7 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/AdminStmtTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/AdminStmtTest.java @@ -69,7 +69,7 @@ public static void beforeClass() throws Exception { // create database String createDbStmtStr = "create database test;"; CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseStmtWithNewParser(createDbStmtStr, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().createDb(createDbStmt.getFullDbName()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createDb(createDbStmt.getFullDbName()); String sql = "CREATE TABLE test.tbl1 (\n" + " `id` int(11) NULL COMMENT \"\",\n" + @@ -112,7 +112,7 @@ public void testAdminSetReplicaStatus() throws Exception { + backendId + "', 'status' = 'bad');"; AdminSetReplicaStatusStmt stmt = (AdminSetReplicaStatusStmt) UtFrameUtils.parseStmtWithNewParser(adminStmt, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().setReplicaStatus(stmt); + GlobalStateMgr.getCurrentState().getTabletManager().setReplicaStatus(stmt); replica = GlobalStateMgr.getCurrentState().getTabletInvertedIndex().getReplica(tabletId, backendId); Assert.assertTrue(replica.isBad()); @@ -120,7 +120,7 @@ public void testAdminSetReplicaStatus() throws Exception { adminStmt = "admin set replica status properties ('tablet_id' = '" + tabletId + "', 'backend_id' = '" + backendId + "', 'status' = 'ok');"; stmt = (AdminSetReplicaStatusStmt) UtFrameUtils.parseStmtWithNewParser(adminStmt, connectContext); - GlobalStateMgr.getCurrentState().getLocalMetastore().setReplicaStatus(stmt); + GlobalStateMgr.getCurrentState().getTabletManager().setReplicaStatus(stmt); replica = GlobalStateMgr.getCurrentState().getTabletInvertedIndex().getReplica(tabletId, backendId); Assert.assertFalse(replica.isBad()); } diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/LocalTabletTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/LocalTabletTest.java index 8813f55d16fff..d3e14224b7a53 100644 --- a/fe/fe-core/src/test/java/com/starrocks/catalog/LocalTabletTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/LocalTabletTest.java @@ -50,11 +50,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; import java.util.List; public class LocalTabletTest { @@ -155,25 +150,6 @@ public void deleteReplicaTest() { @Test public void testSerialization() throws Exception { - File file = new File("./olapTabletTest"); - file.createNewFile(); - DataOutputStream dos = new DataOutputStream(new FileOutputStream(file)); - tablet.write(dos); - dos.flush(); - dos.close(); - - // Read an object from file - DataInputStream dis = new DataInputStream(new FileInputStream(file)); - LocalTablet rTablet1 = LocalTablet.read(dis); - Assert.assertEquals(1, rTablet1.getId()); - Assert.assertEquals(3, rTablet1.getImmutableReplicas().size()); - Assert.assertEquals(rTablet1.getImmutableReplicas().get(0).getVersion(), - rTablet1.getImmutableReplicas().get(1).getVersion()); - - Assert.assertTrue(rTablet1.equals(tablet)); - Assert.assertTrue(rTablet1.equals(rTablet1)); - Assert.assertFalse(rTablet1.equals(this)); - LocalTablet tablet2 = new LocalTablet(1); Replica replica1 = new Replica(1L, 1L, 100L, 0, 200000L, 3000L, ReplicaState.NORMAL, 0, 0); Replica replica2 = new Replica(2L, 2L, 100L, 0, 200001L, 3001L, ReplicaState.NORMAL, 0, 0); @@ -190,9 +166,6 @@ public void testSerialization() throws Exception { tablet3.addReplica(new Replica(4L, 4L, 100L, 0, 200002L, 3002L, ReplicaState.NORMAL, 0, 0)); Assert.assertFalse(tablet3.equals(tablet)); - dis.close(); - file.delete(); - // Read an object from json String jsonStr = GsonUtils.GSON.toJson(tablet); LocalTablet jTablet = GsonUtils.GSON.fromJson(jsonStr, LocalTablet.class); diff --git a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java index cd551d08a6e55..08a98c19a3fd5 100644 --- a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java @@ -41,6 +41,8 @@ import com.starrocks.common.util.UUIDUtil; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.leader.ReportHandler; +import com.starrocks.meta.StarRocksMeta; import com.starrocks.persist.EditLog; import com.starrocks.persist.ModifyPartitionInfo; import com.starrocks.persist.PhysicalPartitionPersistInfoV2; @@ -96,12 +98,12 @@ public void testGetNewPartitionsFromPartitions() throws DdlException { Partition sourcePartition = olapTable.getPartition("t1"); List sourcePartitionIds = Lists.newArrayList(sourcePartition.getId()); List tmpPartitionIds = Lists.newArrayList(connectContext.getGlobalStateMgr().getNextId()); - LocalMetastore localMetastore = connectContext.getGlobalStateMgr().getLocalMetastore(); + StarRocksMeta starRocksMeta = connectContext.getGlobalStateMgr().getStarRocksMeta(); Map origPartitions = Maps.newHashMap(); - OlapTable copiedTable = localMetastore.getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions); + OlapTable copiedTable = starRocksMeta.getCopiedTable(db, olapTable, sourcePartitionIds, origPartitions); Assert.assertEquals(olapTable.getName(), copiedTable.getName()); Set tabletIdSet = Sets.newHashSet(); - List newPartitions = localMetastore.getNewPartitionsFromPartitions(db, + List newPartitions = starRocksMeta.getNewPartitionsFromPartitions(db, olapTable, sourcePartitionIds, origPartitions, copiedTable, "_100", tabletIdSet, tmpPartitionIds, null, WarehouseManager.DEFAULT_WAREHOUSE_ID); Assert.assertEquals(sourcePartitionIds.size(), newPartitions.size()); @@ -149,7 +151,7 @@ public void logModifyPartition(ModifyPartitionInfo info) { }; LocalMetastore localMetastore = connectContext.getGlobalStateMgr().getLocalMetastore(); - localMetastore.getPartitionIdToStorageMediumMap(); + ReportHandler.getPartitionIdToStorageMediumMap(); // Clean test.mv1, avoid its refreshment affecting other cases in this testsuite. starRocksAssert.dropMaterializedView("test.mv1"); } diff --git a/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java b/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java index ba7b834512831..9ee6977a67a30 100644 --- a/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java +++ b/fe/fe-core/src/test/java/com/starrocks/utframe/StarRocksAssert.java @@ -174,7 +174,7 @@ public StarRocksAssert withDatabase(String dbName) throws Exception { DropDbStmt dropDbStmt = (DropDbStmt) UtFrameUtils.parseStmtWithNewParser("drop database if exists `" + dbName + "`;", ctx); try { - GlobalStateMgr.getCurrentState().getLocalMetastore().dropDb(dropDbStmt.getDbName(), dropDbStmt.isForceDrop()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropDb(dropDbStmt.getDbName(), dropDbStmt.isForceDrop()); } catch (MetaNotFoundException e) { if (!dropDbStmt.isSetIfExists()) { ErrorReport.reportDdlException(ErrorCode.ERR_DB_DROP_EXISTS, dbName); @@ -183,7 +183,7 @@ public StarRocksAssert withDatabase(String dbName) throws Exception { CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseStmtWithNewParser("create database `" + dbName + "`;", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().createDb(createDbStmt.getFullDbName()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createDb(createDbStmt.getFullDbName()); return this; } @@ -192,7 +192,7 @@ public StarRocksAssert createDatabaseIfNotExists(String dbName) throws Exception CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseStmtWithNewParser("create database if not exists `" + dbName + "`;", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().createDb(createDbStmt.getFullDbName()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createDb(createDbStmt.getFullDbName()); } catch (AlreadyExistsException e) { // ignore } @@ -216,7 +216,7 @@ public StarRocksAssert withUser(String user) throws Exception { public StarRocksAssert withDatabaseWithoutAnalyze(String dbName) throws Exception { CreateDbStmt dbStmt = new CreateDbStmt(false, dbName); - GlobalStateMgr.getCurrentState().getLocalMetastore().createDb(dbStmt.getFullDbName()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createDb(dbStmt.getFullDbName()); return this; } @@ -339,7 +339,7 @@ public static void utCreateTableWithRetry(CreateTableStmt createTableStmt, Conne createTableStmtCopied = (CreateTableStmt) UtFrameUtils.parseStmtWithNewParser( createTableStmt.getOrigStmt().originStmt, ctx); } - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(createTableStmtCopied); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(createTableStmtCopied); }, "Create Table", 3); } @@ -350,7 +350,7 @@ public static void utDropTableWithRetry(DropTableStmt dropTableStmt, ConnectCont dropTableStmtCopied = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser( dropTableStmt.getOrigStmt().originStmt, ctx); } - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmtCopied); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmtCopied); }, "Drop Table", 3); } @@ -571,13 +571,13 @@ public StarRocksAssert useTable(String table) { } public Table getTable(String dbName, String tableName) { - return ctx.getGlobalStateMgr().getLocalMetastore().mayGetDb(dbName) + return ctx.getGlobalStateMgr().getStarRocksMeta().mayGetDb(dbName) .map(db -> GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), tableName)) .orElse(null); } public MaterializedView getMv(String dbName, String tableName) { - return (MaterializedView) ctx.getGlobalStateMgr().getLocalMetastore() + return (MaterializedView) ctx.getGlobalStateMgr().getStarRocksMeta() .mayGetDb(dbName) .map(db -> GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), tableName)) .orElse(null); @@ -591,7 +591,7 @@ public StarRocksAssert withSingleReplicaTable(String sql) throws Exception { } CreateTableStmt createTableStmt = (CreateTableStmt) statementBase; createTableStmt.getProperties().put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "1"); - GlobalStateMgr.getCurrentState().getLocalMetastore().createTable(createTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createTable(createTableStmt); } catch (Exception e) { LOG.warn("create table failed, sql:{}", sql, e); throw e; @@ -621,7 +621,7 @@ public void withAsyncMv( if (isOnlySingleReplica) { createMaterializedViewStatement.getProperties().put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "1"); } - GlobalStateMgr.getCurrentState().getLocalMetastore().createMaterializedView(createMaterializedViewStatement); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createMaterializedView(createMaterializedViewStatement); if (isRefresh) { new MockUp() { @Mock @@ -658,7 +658,7 @@ public void handleDMLStmt(ExecPlan execPlan, DmlStmt stmt) throws Exception { public StarRocksAssert withView(String sql) throws Exception { CreateViewStmt createTableStmt = (CreateViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().createView(createTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createView(createTableStmt); return this; } @@ -679,7 +679,7 @@ public StarRocksAssert withView(String sql, ExceptionRunnable action) throws Exc public StarRocksAssert dropView(String viewName) throws Exception { DropTableStmt dropViewStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser("drop view " + viewName + ";", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropViewStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropViewStmt); return this; } @@ -693,13 +693,13 @@ public StarRocksAssert dropCatalog(String catalogName) throws Exception { public StarRocksAssert dropDatabase(String dbName) throws Exception { DropDbStmt dropDbStmt = (DropDbStmt) UtFrameUtils.parseStmtWithNewParser("drop database " + dbName + ";", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropDb(dropDbStmt.getDbName(), dropDbStmt.isForceDrop()); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropDb(dropDbStmt.getDbName(), dropDbStmt.isForceDrop()); return this; } public StarRocksAssert alterMvProperties(String sql) throws Exception { AlterMaterializedViewStmt alterMvStmt = (AlterMaterializedViewStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterMaterializedView(alterMvStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterMaterializedView(alterMvStmt); return this; } @@ -708,14 +708,14 @@ public StarRocksAssert alterTableProperties(String sql) throws Exception { Assert.assertFalse(alterTableStmt.getAlterClauseList().isEmpty()); Assert.assertTrue(alterTableStmt.getAlterClauseList().get(0) instanceof ModifyTablePropertiesClause); Analyzer.analyze(alterTableStmt, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); return this; } public StarRocksAssert alterTable(String sql) throws Exception { AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); Analyzer.analyze(alterTableStmt, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); return this; } @@ -729,7 +729,7 @@ public StarRocksAssert dropTables(List tableNames) throws Exception { public StarRocksAssert dropTable(String tableName) throws Exception { DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseStmtWithNewParser("drop table " + tableName + ";", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropTable(dropTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropTable(dropTableStmt); return this; } @@ -744,7 +744,7 @@ public StarRocksAssert dropTemporaryTable(String tableName, boolean ifExists) th public StarRocksAssert dropMaterializedView(String materializedViewName) throws Exception { DropMaterializedViewStmt dropMaterializedViewStmt = (DropMaterializedViewStmt) UtFrameUtils. parseStmtWithNewParser("drop materialized view if exists " + materializedViewName + ";", ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().dropMaterializedView(dropMaterializedViewStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().dropMaterializedView(dropMaterializedViewStmt); return this; } @@ -883,7 +883,7 @@ public StarRocksAssert withMaterializedView(String sql, if (isOnlySingleReplica) { createMaterializedViewStmt.getProperties().put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "1"); } - GlobalStateMgr.getCurrentState().getLocalMetastore().createMaterializedView(createMaterializedViewStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createMaterializedView(createMaterializedViewStmt); checkAlterJob(); } else { Preconditions.checkState(stmt instanceof CreateMaterializedViewStatement); @@ -891,7 +891,7 @@ public StarRocksAssert withMaterializedView(String sql, if (isOnlySingleReplica) { createMaterializedViewStatement.getProperties().put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "1"); } - GlobalStateMgr.getCurrentState().getLocalMetastore().createMaterializedView(createMaterializedViewStatement); + GlobalStateMgr.getCurrentState().getStarRocksMeta().createMaterializedView(createMaterializedViewStatement); if (isRefresh) { String mvName = createMaterializedViewStatement.getTableName().getTbl(); refreshMvPartition(String.format("refresh materialized view %s", mvName)); @@ -1014,7 +1014,7 @@ public void updateTablePartitionVersion(String dbName, String tableName, long ve // Add rollup public StarRocksAssert withRollup(String sql) throws Exception { AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); - GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + GlobalStateMgr.getCurrentState().getStarRocksMeta().alterTable(ctx, alterTableStmt); checkAlterJob(); return this; }