diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index 3b6d487dddc87a..903b7569e6000c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -868,10 +868,7 @@ public boolean equals(Object obj) { && isKey == other.isKey && isAllowNull == other.isAllowNull && isAutoInc == other.isAutoInc - && getDataType().equals(other.getDataType()) - && getStrLen() == other.getStrLen() - && getPrecision() == other.getPrecision() - && getScale() == other.getScale() + && Objects.equals(type, other.type) && Objects.equals(comment, other.comment) && visible == other.visible && Objects.equals(children, other.children) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index c1fcc85125e5ee..8c4df2ee8aeb32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -198,7 +198,7 @@ public void addTaskResult(MTMVTask task, MTMVRelation relation, // to connection issues such as S3, so it is directly set to null if (!isReplay) { // shouldn't do this while holding mvWriteLock - mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true); + mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true, true); } } catch (Throwable e) { mtmvCache = null; @@ -320,7 +320,7 @@ public MTMVCache getOrGenerateCache(ConnectContext connectionContext) throws Ana MTMVCache mtmvCache; try { // Should new context with ADMIN user - mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true); + mtmvCache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this), true, false); } finally { connectionContext.setThreadLocalInfo(); } @@ -385,7 +385,7 @@ public Pair>, Map> calculateDoublyPartit Map baseToMv = Maps.newHashMap(); Map> relatedPartitionDescs = MTMVPartitionUtil .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItemsWithoutLock(); + Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index 4f85a989e46b5f..02c6f9f5276e5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -3051,23 +3051,19 @@ public Map getAndCopyPartitionItems() throws AnalysisExce "get table read lock timeout, database=" + getQualifiedDbName() + ",table=" + getName()); } try { - return getAndCopyPartitionItemsWithoutLock(); + Map res = Maps.newHashMap(); + for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { + Partition partition = idToPartition.get(entry.getKey()); + if (partition != null) { + res.put(partition.getName(), entry.getValue()); + } + } + return res; } finally { readUnlock(); } } - public Map getAndCopyPartitionItemsWithoutLock() throws AnalysisException { - Map res = Maps.newHashMap(); - for (Entry entry : getPartitionInfo().getIdToItem(false).entrySet()) { - Partition partition = idToPartition.get(entry.getKey()); - if (partition != null) { - res.put(partition.getName(), entry.getValue()); - } - } - return res; - } - @Override public List getPartitionColumns(Optional snapshot) { return getPartitionColumns(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index 8f9594e82c5c33..2efb310657ae27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -210,7 +210,6 @@ default Map getConstraintsMapUnsafe() { } default Set getForeignKeyConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(ForeignKeyConstraint.class::isInstance) @@ -218,24 +217,18 @@ default Set getForeignKeyConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } default Map getConstraintsMap() { - readLock(); try { return ImmutableMap.copyOf(getConstraintsMapUnsafe()); } catch (Exception ignored) { return ImmutableMap.of(); - } finally { - readUnlock(); } } default Set getPrimaryKeyConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(PrimaryKeyConstraint.class::isInstance) @@ -243,13 +236,10 @@ default Set getPrimaryKeyConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } default Set getUniqueConstraints() { - readLock(); try { return getConstraintsMapUnsafe().values().stream() .filter(UniqueConstraint.class::isInstance) @@ -257,8 +247,6 @@ default Set getUniqueConstraints() { .collect(ImmutableSet.toImmutableSet()); } catch (Exception ignored) { return ImmutableSet.of(); - } finally { - readUnlock(); } } @@ -277,34 +265,24 @@ default void checkConstraintNotExistenceUnsafe(String name, Constraint primaryKe } default void addUniqueConstraint(String name, ImmutableList columns, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - UniqueConstraint uniqueConstraint = new UniqueConstraint(name, ImmutableSet.copyOf(columns)); - checkConstraintNotExistenceUnsafe(name, uniqueConstraint, constraintMap); - constraintMap.put(name, uniqueConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(uniqueConstraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + UniqueConstraint uniqueConstraint = new UniqueConstraint(name, ImmutableSet.copyOf(columns)); + checkConstraintNotExistenceUnsafe(name, uniqueConstraint, constraintMap); + constraintMap.put(name, uniqueConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(uniqueConstraint, this)); } } default void addPrimaryKeyConstraint(String name, ImmutableList columns, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint(name, ImmutableSet.copyOf(columns)); - checkConstraintNotExistenceUnsafe(name, primaryKeyConstraint, constraintMap); - constraintMap.put(name, primaryKeyConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(primaryKeyConstraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + PrimaryKeyConstraint primaryKeyConstraint = new PrimaryKeyConstraint(name, ImmutableSet.copyOf(columns)); + checkConstraintNotExistenceUnsafe(name, primaryKeyConstraint, constraintMap); + constraintMap.put(name, primaryKeyConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(primaryKeyConstraint, this)); } } @@ -323,26 +301,19 @@ default PrimaryKeyConstraint tryGetPrimaryKeyForForeignKeyUnsafe( default void addForeignConstraint(String name, ImmutableList columns, TableIf referencedTable, ImmutableList referencedColumns, boolean replay) { - writeLock(); - referencedTable.writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - ForeignKeyConstraint foreignKeyConstraint = - new ForeignKeyConstraint(name, columns, referencedTable, referencedColumns); - checkConstraintNotExistenceUnsafe(name, foreignKeyConstraint, constraintMap); - PrimaryKeyConstraint requirePrimaryKeyName = new PrimaryKeyConstraint(name, - foreignKeyConstraint.getReferencedColumnNames()); - PrimaryKeyConstraint primaryKeyConstraint = - tryGetPrimaryKeyForForeignKeyUnsafe(requirePrimaryKeyName, referencedTable); - primaryKeyConstraint.addForeignTable(this); - constraintMap.put(name, foreignKeyConstraint); - if (!replay) { - Env.getCurrentEnv().getEditLog().logAddConstraint( - new AlterConstraintLog(foreignKeyConstraint, this)); - } - } finally { - referencedTable.writeUnlock(); - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + ForeignKeyConstraint foreignKeyConstraint = + new ForeignKeyConstraint(name, columns, referencedTable, referencedColumns); + checkConstraintNotExistenceUnsafe(name, foreignKeyConstraint, constraintMap); + PrimaryKeyConstraint requirePrimaryKeyName = new PrimaryKeyConstraint(name, + foreignKeyConstraint.getReferencedColumnNames()); + PrimaryKeyConstraint primaryKeyConstraint = + tryGetPrimaryKeyForForeignKeyUnsafe(requirePrimaryKeyName, referencedTable); + primaryKeyConstraint.addForeignTable(this); + constraintMap.put(name, foreignKeyConstraint); + if (!replay) { + Env.getCurrentEnv().getEditLog().logAddConstraint( + new AlterConstraintLog(foreignKeyConstraint, this)); } } @@ -378,40 +349,31 @@ default void replayDropConstraint(String name) { } default void dropConstraint(String name, boolean replay) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - if (!constraintMap.containsKey(name)) { - throw new AnalysisException( - String.format("Unknown constraint %s on table %s.", name, this.getName())); - } - Constraint constraint = constraintMap.get(name); - constraintMap.remove(name); - if (constraint instanceof PrimaryKeyConstraint) { - ((PrimaryKeyConstraint) constraint).getForeignTables() - .forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint)); - } - if (!replay) { - Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(constraint, this)); - } - } finally { - writeUnlock(); + Map constraintMap = getConstraintsMapUnsafe(); + if (!constraintMap.containsKey(name)) { + throw new AnalysisException( + String.format("Unknown constraint %s on table %s.", name, this.getName())); + } + Constraint constraint = constraintMap.get(name); + constraintMap.remove(name); + if (constraint instanceof PrimaryKeyConstraint) { + ((PrimaryKeyConstraint) constraint).getForeignTables() + .forEach(t -> t.dropFKReferringPK(this, (PrimaryKeyConstraint) constraint)); + } + if (!replay) { + Env.getCurrentEnv().getEditLog().logDropConstraint(new AlterConstraintLog(constraint, this)); } } default void dropFKReferringPK(TableIf table, PrimaryKeyConstraint constraint) { - writeLock(); - try { - Map constraintMap = getConstraintsMapUnsafe(); - Set fkName = constraintMap.entrySet().stream() - .filter(e -> e.getValue() instanceof ForeignKeyConstraint - && ((ForeignKeyConstraint) e.getValue()).isReferringPK(table, constraint)) - .map(Entry::getKey) - .collect(Collectors.toSet()); - fkName.forEach(constraintMap::remove); - } finally { - writeUnlock(); - } + Map constraintMap = getConstraintsMapUnsafe(); + Set fkName = constraintMap.entrySet().stream() + .filter(e -> e.getValue() instanceof ForeignKeyConstraint + && ((ForeignKeyConstraint) e.getValue()).isReferringPK(table, constraint)) + .map(Entry::getKey) + .collect(Collectors.toSet()); + fkName.forEach(constraintMap::remove); + } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java index 87383363c0010f..e81e52e4ef151a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/NereidsSqlCacheManager.java @@ -224,6 +224,10 @@ private Optional tryParseSqlWithoutCheckVariable( SqlCacheContext sqlCacheContext, UserIdentity currentUserIdentity) { Env env = connectContext.getEnv(); + if (!tryLockTables(connectContext, env, sqlCacheContext)) { + return invalidateCache(key); + } + // check table and view and their columns authority if (privilegeChanged(connectContext, env, sqlCacheContext)) { return invalidateCache(key); @@ -377,16 +381,38 @@ private boolean dataMaskPoliciesChanged( return false; } - private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { + /** + * Execute table locking operations in ascending order of table IDs. + * + * @return true if obtain all tables lock. + */ + private boolean tryLockTables(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { StatementContext currentStatementContext = connectContext.getStatementContext(); + for (FullTableName fullTableName : sqlCacheContext.getUsedTables()) { + TableIf tableIf = findTableIf(env, fullTableName); + if (tableIf == null) { + return false; + } + currentStatementContext.getTables().put(fullTableName.toList(), tableIf); + } + for (FullTableName fullTableName : sqlCacheContext.getUsedViews().keySet()) { + TableIf tableIf = findTableIf(env, fullTableName); + if (tableIf == null) { + return false; + } + currentStatementContext.getTables().put(fullTableName.toList(), tableIf); + } + currentStatementContext.lock(); + return true; + } + + private boolean privilegeChanged(ConnectContext connectContext, Env env, SqlCacheContext sqlCacheContext) { for (Entry> kv : sqlCacheContext.getCheckPrivilegeTablesOrViews().entrySet()) { Set usedColumns = kv.getValue(); TableIf tableIf = findTableIf(env, kv.getKey()); if (tableIf == null) { return true; } - // release when close statementContext - currentStatementContext.addTableReadLock(tableIf); try { UserAuthentication.checkPermission(tableIf, connectContext, usedColumns); } catch (Throwable t) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java b/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java index 7a6f0db5938b23..de825fbdb3ac23 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/lock/MonitoredReentrantReadWriteLock.java @@ -17,6 +17,12 @@ package org.apache.doris.common.lock; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.qe.ConnectContext; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -24,6 +30,8 @@ * monitoring capabilities for read and write locks. */ public class MonitoredReentrantReadWriteLock extends ReentrantReadWriteLock { + + private static final Logger LOG = LogManager.getLogger(MonitoredReentrantReadWriteLock.class); // Monitored read and write lock instances private final ReadLock readLock = new ReadLock(this); private final WriteLock writeLock = new WriteLock(this); @@ -97,6 +105,11 @@ protected WriteLock(ReentrantReadWriteLock lock) { public void lock() { super.lock(); monitor.afterLock(); + if (isFair() && getReadHoldCount() > 0) { + LOG.warn(" read lock count is {}, write lock count is {}, stack is {}, query id is {}", + getReadHoldCount(), getWriteHoldCount(), Thread.currentThread().getStackTrace(), + ConnectContext.get() == null ? "" : DebugUtil.printId(ConnectContext.get().queryId())); + } } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java index 0d7c48ee319b53..6b5a63a49e4005 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java @@ -35,6 +35,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.Type; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; @@ -44,9 +45,12 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ListComparator; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.OrderByPair; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVPartitionUtil; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; @@ -59,6 +63,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -250,22 +255,38 @@ private List, TRow>> getPartitionInfosInrernal() throws An List, TRow>> partitionInfos = new ArrayList, TRow>>(); Map> partitionsUnSyncTables = null; String mtmvPartitionSyncErrorMsg = null; + + List needLocked = Lists.newArrayList(); + needLocked.add(olapTable); if (olapTable instanceof MTMV) { - try { - partitionsUnSyncTables = MTMVPartitionUtil - .getPartitionsUnSyncTables((MTMV) olapTable); - } catch (AnalysisException e) { - mtmvPartitionSyncErrorMsg = e.getMessage(); + MTMV mtmv = (MTMV) olapTable; + for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTables()) { + try { + TableIf baseTable = MTMVUtil.getTable(baseTableInfo); + needLocked.add(baseTable); + } catch (Exception e) { + // do nothing, ignore not existed table + } } + needLocked.sort(Comparator.comparing(TableIf::getId)); } - olapTable.readLock(); + MetaLockUtils.readLockTables(needLocked); try { + if (olapTable instanceof MTMV) { + try { + partitionsUnSyncTables = MTMVPartitionUtil + .getPartitionsUnSyncTables((MTMV) olapTable); + } catch (AnalysisException e) { + mtmvPartitionSyncErrorMsg = e.getMessage(); + } + } List partitionIds; PartitionInfo tblPartitionInfo = olapTable.getPartitionInfo(); // for range partitions, we return partitions in ascending range order by default. // this is to be consistent with the behaviour before 0.12 - if (tblPartitionInfo.getType() == PartitionType.RANGE || tblPartitionInfo.getType() == PartitionType.LIST) { + if (tblPartitionInfo.getType() == PartitionType.RANGE + || tblPartitionInfo.getType() == PartitionType.LIST) { partitionIds = tblPartitionInfo.getPartitionItemEntryList(isTempPartition, true).stream() .map(Map.Entry::getKey).collect(Collectors.toList()); } else { @@ -402,7 +423,7 @@ private List, TRow>> getPartitionInfosInrernal() throws An partitionInfos.add(Pair.of(partitionInfo, trow)); } } finally { - olapTable.readUnlock(); + MetaLockUtils.readUnlockTables(needLocked); } return partitionInfos; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index a10aca3afd765b..152b2bab3fa6fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -84,6 +84,7 @@ public class SummaryProfile { public static final String WRITE_RESULT_TIME = "Write Result Time"; public static final String PARSE_SQL_TIME = "Parse SQL Time"; + public static final String NEREIDS_LOCK_TABLE_TIME = "Nereids Lock Table Time"; public static final String NEREIDS_ANALYSIS_TIME = "Nereids Analysis Time"; public static final String NEREIDS_REWRITE_TIME = "Nereids Rewrite Time"; public static final String NEREIDS_OPTIMIZE_TIME = "Nereids Optimize Time"; @@ -118,6 +119,7 @@ public class SummaryProfile { // The display order of execution summary items. public static final ImmutableList EXECUTION_SUMMARY_KEYS = ImmutableList.of( PARSE_SQL_TIME, + NEREIDS_LOCK_TABLE_TIME, NEREIDS_ANALYSIS_TIME, NEREIDS_REWRITE_TIME, NEREIDS_OPTIMIZE_TIME, @@ -194,6 +196,8 @@ public class SummaryProfile { private long parseSqlStartTime = -1; private long parseSqlFinishTime = -1; + + private long nereidsLockTableFinishTime = -1; private long nereidsAnalysisFinishTime = -1; private long nereidsRewriteFinishTime = -1; private long nereidsOptimizeFinishTime = -1; @@ -311,6 +315,7 @@ private void updateSummaryProfile(Map infos) { private void updateExecutionSummaryProfile() { executionSummaryProfile.addInfoString(PARSE_SQL_TIME, getPrettyParseSqlTime()); + executionSummaryProfile.addInfoString(NEREIDS_LOCK_TABLE_TIME, getPrettyNereidsLockTableTime()); executionSummaryProfile.addInfoString(NEREIDS_ANALYSIS_TIME, getPrettyNereidsAnalysisTime()); executionSummaryProfile.addInfoString(NEREIDS_REWRITE_TIME, getPrettyNereidsRewriteTime()); executionSummaryProfile.addInfoString(NEREIDS_OPTIMIZE_TIME, getPrettyNereidsOptimizeTime()); @@ -397,6 +402,10 @@ public void setParseSqlFinishTime(long parseSqlFinishTime) { this.parseSqlFinishTime = parseSqlFinishTime; } + public void setNereidsLockTableFinishTime() { + this.nereidsLockTableFinishTime = TimeUtils.getStartTimeMs(); + } + public void setNereidsAnalysisTime() { this.nereidsAnalysisFinishTime = TimeUtils.getStartTimeMs(); } @@ -640,8 +649,12 @@ public String getPrettyParseSqlTime() { return getPrettyTime(parseSqlFinishTime, parseSqlStartTime, TUnit.TIME_MS); } + public String getPrettyNereidsLockTableTime() { + return getPrettyTime(nereidsLockTableFinishTime, parseSqlStartTime, TUnit.TIME_MS); + } + public String getPrettyNereidsAnalysisTime() { - return getPrettyTime(nereidsAnalysisFinishTime, queryBeginTime, TUnit.TIME_MS); + return getPrettyTime(nereidsAnalysisFinishTime, nereidsLockTableFinishTime, TUnit.TIME_MS); } public String getPrettyNereidsRewriteTime() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 3d1d001d61ab4c..a26693cb4110bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -27,6 +27,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; @@ -65,6 +66,7 @@ import java.math.BigDecimal; import java.math.RoundingMode; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -170,13 +172,25 @@ public void run() throws JobException { } // Every time a task is run, the relation is regenerated because baseTables and baseViews may change, // such as deleting a table and creating a view with the same name - this.relation = MTMVPlanUtil.generateMTMVRelation(mtmv, ctx); + Set tablesInPlan = MTMVPlanUtil.getBaseTableFromQuery(mtmv.getQuerySql(), ctx); + this.relation = MTMVPlanUtil.generateMTMVRelation(tablesInPlan, ctx); beforeMTMVRefresh(); - if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - MTMVPartitionUtil.alignMvPartition(mtmv); + + List tableIfs = Lists.newArrayList(tablesInPlan); + tableIfs.sort(Comparator.comparing(TableIf::getId)); + + MTMVRefreshContext context; + // lock table order by id to avoid deadlock + MetaLockUtils.readLockTables(tableIfs); + try { + if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { + MTMVPartitionUtil.alignMvPartition(mtmv); + } + context = MTMVRefreshContext.buildContext(mtmv); + this.needRefreshPartitions = calculateNeedRefreshPartitions(context); + } finally { + MetaLockUtils.readUnlockTables(tableIfs); } - MTMVRefreshContext context = MTMVRefreshContext.buildContext(mtmv); - this.needRefreshPartitions = calculateNeedRefreshPartitions(context); this.refreshMode = generateRefreshMode(needRefreshPartitions); if (refreshMode == MTMVTaskRefreshMode.NOT_REFRESH) { return; @@ -191,7 +205,7 @@ public void run() throws JobException { int start = i * refreshPartitionNum; int end = start + refreshPartitionNum; Set execPartitionNames = Sets.newHashSet(needRefreshPartitions - .subList(start, end > needRefreshPartitions.size() ? needRefreshPartitions.size() : end)); + .subList(start, Math.min(end, needRefreshPartitions.size()))); // need get names before exec Map execPartitionSnapshots = MTMVPartitionUtil .generatePartitionSnapshots(context, relation.getBaseTablesOneLevel(), execPartitionNames); @@ -201,7 +215,7 @@ public void run() throws JobException { } } catch (Throwable e) { if (getStatus() == TaskStatus.RUNNING) { - LOG.warn("run task failed: ", e.getMessage()); + LOG.warn("run task failed: {}", e.getMessage()); throw new JobException(e.getMessage(), e); } else { // if status is not `RUNNING`,maybe the task was canceled, therefore, it is a normal situation diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java index 9b4104703079fc..44183c38fef834 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseTableInfo.java @@ -26,11 +26,14 @@ import org.apache.doris.datasource.InternalCatalog; import com.google.common.base.Objects; +import com.google.common.collect.Lists; import com.google.gson.annotations.SerializedName; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.List; + public class BaseTableInfo { private static final Logger LOG = LogManager.getLogger(BaseTableInfo.class); @@ -167,4 +170,8 @@ public void compatible(CatalogMgr catalogMgr) { LOG.warn("MTMV compatible failed, ctlId: {}, dbId: {}, tableId: {}", ctlId, dbId, tableId, e); } } + + public List toList() { + return Lists.newArrayList(getCtlName(), getDbName(), getTableName()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java index d3d7f1ad6ebbf5..b185000c14897e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java @@ -87,23 +87,31 @@ public StructInfo getStructInfo() { return structInfo; } - public static MTMVCache from(MTMV mtmv, ConnectContext connectContext, boolean needCost) { + public static MTMVCache from(MTMV mtmv, ConnectContext connectContext, boolean needCost, boolean needLock) { StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(mtmv.getQuerySql(), 0)); + if (needLock) { + mvSqlStatementContext.setNeedLockTables(false); + } if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); } LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(mtmv.getQuerySql()); NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); - - // Can not convert to table sink, because use the same column from different table when self join - // the out slot is wrong - if (needCost) { - // Only in mv rewrite, we need plan with eliminated cost which is used for mv chosen - planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); - } else { - // No need cost for performance - planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); + boolean originalRewriteFlag = connectContext.getSessionVariable().enableMaterializedViewRewrite; + connectContext.getSessionVariable().enableMaterializedViewRewrite = false; + try { + // Can not convert to table sink, because use the same column from different table when self join + // the out slot is wrong + if (needCost) { + // Only in mv rewrite, we need plan with eliminated cost which is used for mv chosen + planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + } else { + // No need cost for performance + planner.planWithLock(unboundMvPlan, PhysicalProperties.ANY, ExplainLevel.REWRITTEN_PLAN); + } + } finally { + connectContext.getSessionVariable().enableMaterializedViewRewrite = originalRewriteFlag; } Plan originPlan = planner.getCascadesContext().getRewritePlan(); // Eliminate result sink because sink operator is useless in query rewrite by materialized view @@ -128,6 +136,6 @@ public Plan visitLogicalResultSink(LogicalResultSink logicalResu new BitSet()); return new MTMVCache(mvPlan, originPlan, planner.getAnalyzedPlan(), needCost ? planner.getCascadesContext().getMemo().getRoot().getStatistics() : null, - structInfoOptional.orElseGet(() -> null)); + structInfoOptional.orElse(null)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java index 2c03ad16176fea..a9dee132f64b12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVJobManager.java @@ -104,17 +104,18 @@ private void setManualJobConfig(JobExecutionConfiguration jobExecutionConfigurat private void setScheduleJobConfig(JobExecutionConfiguration jobExecutionConfiguration, MTMV mtmv) { jobExecutionConfiguration.setExecuteType(JobExecuteType.RECURRING); + MTMVRefreshInfo refreshMTMVInfo = mtmv.getRefreshInfo(); TimerDefinition timerDefinition = new TimerDefinition(); timerDefinition - .setInterval(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getInterval()); + .setInterval(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getInterval()); timerDefinition - .setIntervalUnit(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); + .setIntervalUnit(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getTimeUnit()); if (!StringUtils - .isEmpty(mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { + .isEmpty(refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getStartTime())) { timerDefinition.setStartTimeMs(TimeUtils.timeStringToLong( - mtmv.getRefreshInfo().getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); + refreshMTMVInfo.getRefreshTriggerInfo().getIntervalTrigger().getStartTime())); } - if (mtmv.getRefreshInfo().getBuildMode().equals(BuildMode.IMMEDIATE)) { + if (refreshMTMVInfo.getBuildMode().equals(BuildMode.IMMEDIATE)) { jobExecutionConfiguration.setImmediate(true); } jobExecutionConfiguration.setTimerDefinition(timerDefinition); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index e2775970ab6ab8..94cd34195a0f62 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -535,11 +535,12 @@ private static Map getPartitionVersions(MTMV mtmv) throws Analysis private static Map getTableVersions(MTMV mtmv) { Map res = Maps.newHashMap(); - if (mtmv.getRelation() == null || mtmv.getRelation().getBaseTablesOneLevel() == null) { + MTMVRelation relation = mtmv.getRelation(); + if (relation == null || relation.getBaseTablesOneLevel() == null) { return res; } List olapTables = Lists.newArrayList(); - for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTablesOneLevel()) { + for (BaseTableInfo baseTableInfo : relation.getBaseTablesOneLevel()) { TableIf table = null; try { table = MTMVUtil.getTable(baseTableInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 5761b41171e644..6c9f208d3d8206 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -33,11 +33,8 @@ import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableSet; @@ -101,41 +98,27 @@ private static void setCatalogAndDb(ConnectContext ctx, MTMV mtmv) { ctx.setDatabase(databaseIf.get().getFullName()); } - public static MTMVRelation generateMTMVRelation(MTMV mtmv, ConnectContext ctx) { - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - Plan plan = getAnalyzePlanBySql(mtmv.getQuerySql(), ctx); - return generateMTMVRelation(plan, ctx); - } - - public static MTMVRelation generateMTMVRelation(Plan plan, ConnectContext connectContext) { - return new MTMVRelation(getBaseTables(plan, true, connectContext), - getBaseTables(plan, false, connectContext), getBaseViews(plan)); - } - - private static Set getBaseTables(Plan plan, boolean expand, ConnectContext connectContext) { - TableCollectorContext collectorContext = - new TableCollector.TableCollectorContext( - com.google.common.collect.Sets - .newHashSet(TableType.values()), expand, connectContext); - plan.accept(TableCollector.INSTANCE, collectorContext); - Set collectedTables = collectorContext.getCollectedTables(); - return transferTableIfToInfo(collectedTables); - } - - private static Set getBaseViews(Plan plan) { - return Sets.newHashSet(); - } - - private static Set transferTableIfToInfo(Set tables) { - Set result = com.google.common.collect.Sets.newHashSet(); - for (TableIf table : tables) { - result.add(new BaseTableInfo(table)); + public static MTMVRelation generateMTMVRelation(Set tablesInPlan, ConnectContext ctx) { + Set oneLevelTables = Sets.newHashSet(); + Set allLevelTables = Sets.newHashSet(); + Set oneLevelViews = Sets.newHashSet(); + for (TableIf table : tablesInPlan) { + BaseTableInfo baseTableInfo = new BaseTableInfo(table); + if (table.getType() == TableType.VIEW) { + // TODO reopen it after we support mv on view + // oneLevelViews.add(baseTableInfo); + } else { + oneLevelTables.add(baseTableInfo); + allLevelTables.add(baseTableInfo); + if (table instanceof MTMV) { + allLevelTables.addAll(((MTMV) table).getRelation().getBaseTables()); + } + } } - return result; + return new MTMVRelation(allLevelTables, oneLevelTables, oneLevelViews); } - private static Plan getAnalyzePlanBySql(String querySql, ConnectContext ctx) { + public static Set getBaseTableFromQuery(String querySql, ConnectContext ctx) { List statements; try { statements = new NereidsParser().parseSQL(querySql); @@ -145,12 +128,15 @@ private static Plan getAnalyzePlanBySql(String querySql, ConnectContext ctx) { StatementBase parsedStmt = statements.get(0); LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStmt).getLogicalPlan(); StatementContext original = ctx.getStatementContext(); - ctx.setStatementContext(new StatementContext()); - try { - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - return planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.ANALYZED_PLAN); - } finally { - ctx.setStatementContext(original); + try (StatementContext tempCtx = new StatementContext()) { + ctx.setStatementContext(tempCtx); + try { + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.planWithLock(logicalPlan, PhysicalProperties.ANY, ExplainLevel.ANALYZED_PLAN); + return Sets.newHashSet(ctx.getStatementContext().getTables().values()); + } finally { + ctx.setStatementContext(original); + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java index 436427526ba08b..f8f92e25d38d65 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java @@ -59,8 +59,8 @@ public class MTMVRelationManager implements MTMVHookService { // create mv2 as select * from mv1; // `tableMTMVs` will have 3 pair: table1 ==> mv1,mv1==>mv2, table1 ==> mv2 // `tableMTMVsOneLevel` will have 2 pair: table1 ==> mv1,mv1==>mv2 - private Map> tableMTMVs = Maps.newConcurrentMap(); - private Map> tableMTMVsOneLevel = Maps.newConcurrentMap(); + private final Map> tableMTMVs = Maps.newConcurrentMap(); + private final Map> tableMTMVsOneLevel = Maps.newConcurrentMap(); public Set getMtmvsByBaseTable(BaseTableInfo table) { return tableMTMVs.getOrDefault(table, ImmutableSet.of()); @@ -98,6 +98,23 @@ public Set getAvailableMTMVs(List tableInfos, ConnectContex return res; } + /** + * get all mtmv related to tableInfos. + */ + public Set getAllMTMVs(List tableInfos) { + Set mtmvs = Sets.newLinkedHashSet(); + Set mvInfos = getMTMVInfos(tableInfos); + for (BaseTableInfo tableInfo : mvInfos) { + try { + mtmvs.add((MTMV) MTMVUtil.getTable(tableInfo)); + } catch (AnalysisException e) { + // not throw exception to client, just ignore it + LOG.warn("getTable failed: {}", tableInfo.toString(), e); + } + } + return mtmvs; + } + @VisibleForTesting public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) { long currentTimeMillis = System.currentTimeMillis(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java index 7b7d743a36bc8c..ff1b3263d3409e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java @@ -51,8 +51,8 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne return res; } // check mv is normal - if (mtmv.getStatus().getState() != MTMVState.NORMAL - || mtmv.getStatus().getRefreshState() == MTMVRefreshState.INIT) { + MTMVStatus mtmvStatus = mtmv.getStatus(); + if (mtmvStatus.getState() != MTMVState.NORMAL || mtmvStatus.getRefreshState() == MTMVRefreshState.INIT) { return res; } MTMVRefreshContext refreshContext = null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 403d05f8c18f31..258704763909f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -17,20 +17,13 @@ package org.apache.doris.nereids; -import org.apache.doris.catalog.DatabaseIf; -import org.apache.doris.catalog.Env; -import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; -import org.apache.doris.datasource.CatalogIf; import org.apache.doris.nereids.analyzer.Scope; -import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; -import org.apache.doris.nereids.analyzer.UnboundRelation; -import org.apache.doris.nereids.analyzer.UnboundTableSink; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.jobs.Job; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; +import org.apache.doris.nereids.jobs.executor.TableCollector; import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob; import org.apache.doris.nereids.jobs.rewrite.RewriteTopDownJob; import org.apache.doris.nereids.jobs.rewrite.RootPlanTreeRewriteJob.RootRewriteJobContext; @@ -46,7 +39,6 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.exploration.mv.MaterializationContext; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; @@ -54,25 +46,16 @@ import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.RelationId; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; -import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.Statistics; import org.apache.doris.statistics.StatisticsBuilder; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; -import org.apache.commons.collections.MapUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -86,8 +69,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.Stack; -import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -105,7 +86,7 @@ public class CascadesContext implements ScheduleContext { private Memo memo; private final StatementContext statementContext; - private final CTEContext cteContext; + private CTEContext cteContext; private final RuleSet ruleSet; private final JobPool jobPool; private final JobScheduler jobScheduler; @@ -115,7 +96,6 @@ public class CascadesContext implements ScheduleContext { private final RuntimeFilterContext runtimeFilterContext; private final TopnFilterContext topnFilterContext = new TopnFilterContext(); private Optional outerScope = Optional.empty(); - private Map tables = null; private boolean isRewriteRoot; private volatile boolean isTimeout = false; @@ -171,9 +151,6 @@ private CascadesContext(Optional parent, Optional curren } else { this.isEnableExprTrace = false; } - if (parent.isPresent()) { - this.tables = parent.get().tables; - } this.isLeadingDisableJoinReorder = isLeadingDisableJoinReorder; } @@ -247,12 +224,12 @@ public void toMemo() { this.memo = new Memo(getConnectContext(), plan); } - public Analyzer newAnalyzer() { - return newAnalyzer(Optional.empty()); + public TableCollector newTableCollector() { + return new TableCollector(this); } - public Analyzer newAnalyzer(Optional customTableResolver) { - return new Analyzer(this, customTableResolver); + public Analyzer newAnalyzer() { + return new Analyzer(this); } @Override @@ -268,10 +245,6 @@ public void releaseMemo() { this.memo = null; } - public void setTables(List tables) { - this.tables = tables.stream().collect(Collectors.toMap(TableIf::getId, t -> t, (t1, t2) -> t1)); - } - public final ConnectContext getConnectContext() { return statementContext.getConnectContext(); } @@ -346,6 +319,10 @@ public CTEContext getCteContext() { return cteContext; } + public void setCteContext(CTEContext cteContext) { + this.cteContext = cteContext; + } + public void setIsRewriteRoot(boolean isRewriteRoot) { this.isRewriteRoot = isRewriteRoot; } @@ -410,217 +387,6 @@ private CascadesContext execute(Job job) { return this; } - /** - * Extract tables. - */ - public void extractTables(LogicalPlan logicalPlan) { - Set> tableNames = getTables(logicalPlan); - tables = Maps.newHashMap(); - for (List tableName : tableNames) { - try { - TableIf table = getTable(tableName); - tables.put(table.getId(), table); - } catch (Throwable e) { - // IGNORE - } - } - - } - - /** get table by table name, try to get from information from dumpfile first */ - public TableIf getTableInMinidumpCache(String tableName) { - Preconditions.checkState(tables != null, "tables should not be null"); - for (TableIf table : tables.values()) { - if (table.getName().equals(tableName)) { - return table; - } - } - if (getConnectContext().getSessionVariable().isPlayNereidsDump()) { - throw new AnalysisException("Minidump cache can not find table:" + tableName); - } - return null; - } - - public List getTables() { - if (tables == null) { - return null; - } else { - return Lists.newArrayList(tables.values()); - } - } - - private Set> getTables(LogicalPlan logicalPlan) { - final Set> tableNames = new HashSet<>(); - logicalPlan.foreach(p -> { - if (p instanceof LogicalFilter) { - tableNames.addAll(extractTableNamesFromFilter((LogicalFilter) p)); - } else if (p instanceof LogicalCTE) { - tableNames.addAll(extractTableNamesFromCTE((LogicalCTE) p)); - } else if (p instanceof LogicalProject) { - tableNames.addAll(extractTableNamesFromProject((LogicalProject) p)); - } else if (p instanceof LogicalHaving) { - tableNames.addAll(extractTableNamesFromHaving((LogicalHaving) p)); - } else if (p instanceof UnboundOneRowRelation) { - tableNames.addAll(extractTableNamesFromOneRowRelation((UnboundOneRowRelation) p)); - } else { - Set logicalPlans = p.collect( - n -> (n instanceof UnboundRelation || n instanceof UnboundTableSink)); - for (LogicalPlan plan : logicalPlans) { - if (plan instanceof UnboundRelation) { - tableNames.add(((UnboundRelation) plan).getNameParts()); - } else if (plan instanceof UnboundTableSink) { - tableNames.add(((UnboundTableSink) plan).getNameParts()); - } else { - throw new AnalysisException("get tables from plan failed. meet unknown type node " + plan); - } - } - } - }); - return tableNames; - } - - public Map getOrExtractTables(LogicalPlan logicalPlan) { - if (MapUtils.isEmpty(tables)) { - extractTables(logicalPlan); - } - return tables; - } - - private Set> extractTableNamesFromHaving(LogicalHaving having) { - Set subqueryExprs = having.getPredicate() - .collect(SubqueryExpr.class::isInstance); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromOneRowRelation(UnboundOneRowRelation oneRowRelation) { - Set subqueryExprs = oneRowRelation.getProjects().stream() - .>map(p -> p.collect(SubqueryExpr.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromProject(LogicalProject project) { - Set subqueryExprs = project.getProjects().stream() - .>map(p -> p.collect(SubqueryExpr.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toSet()); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromFilter(LogicalFilter filter) { - Set subqueryExprs = filter.getPredicate() - .collect(SubqueryExpr.class::isInstance); - Set> tableNames = new HashSet<>(); - for (SubqueryExpr expr : subqueryExprs) { - LogicalPlan plan = expr.getQueryPlan(); - tableNames.addAll(getTables(plan)); - } - return tableNames; - } - - private Set> extractTableNamesFromCTE(LogicalCTE cte) { - List> subQueryAliases = cte.getAliasQueries(); - Set> tableNames = new HashSet<>(); - for (LogicalSubQueryAlias subQueryAlias : subQueryAliases) { - tableNames.addAll(getTables(subQueryAlias)); - } - return tableNames; - } - - private TableIf getTable(List nameParts) { - switch (nameParts.size()) { - case 1: { // table - String ctlName = getConnectContext().getEnv().getCurrentCatalog().getName(); - String dbName = getConnectContext().getDatabase(); - return getTable(ctlName, dbName, nameParts.get(0), getConnectContext().getEnv()); - } - case 2: { // db.table - String ctlName = getConnectContext().getEnv().getCurrentCatalog().getName(); - String dbName = nameParts.get(0); - return getTable(ctlName, dbName, nameParts.get(1), getConnectContext().getEnv()); - } - case 3: { // catalog.db.table - return getTable(nameParts.get(0), nameParts.get(1), nameParts.get(2), getConnectContext().getEnv()); - } - default: - throw new IllegalStateException("Table name [" + String.join(".", nameParts) + "] is invalid."); - } - } - - /** - * Find table from catalog. - */ - public TableIf getTable(String ctlName, String dbName, String tableName, Env env) { - CatalogIf catalog = env.getCatalogMgr().getCatalog(ctlName); - if (catalog == null) { - throw new RuntimeException("Catalog [" + ctlName + "] does not exist."); - } - DatabaseIf db = catalog.getDbNullable(dbName); - if (db == null) { - throw new RuntimeException("Database [" + dbName + "] does not exist in catalog [" + ctlName + "]."); - } - - TableIf table = db.getTableNullable(tableName); - if (table == null) { - throw new RuntimeException("Table [" + tableName + "] does not exist in database [" + dbName + "]."); - } - return table; - - } - - /** - * Used to lock table - */ - public static class Lock implements AutoCloseable { - - CascadesContext cascadesContext; - private final Stack locked = new Stack<>(); - - /** - * Try to acquire read locks on tables, throw runtime exception once the acquiring for read lock failed. - */ - public Lock(LogicalPlan plan, CascadesContext cascadesContext) { - this.cascadesContext = cascadesContext; - // tables can also be load from dump file - if (cascadesContext.tables == null) { - cascadesContext.extractTables(plan); - } - for (TableIf table : cascadesContext.tables.values()) { - if (!table.needReadLockWhenPlan()) { - continue; - } - if (!table.tryReadLock(1, TimeUnit.MINUTES)) { - close(); - throw new RuntimeException(String.format("Failed to get read lock on table: %s", table.getName())); - } - locked.push(table); - } - } - - @Override - public void close() { - while (!locked.empty()) { - locked.pop().readUnlock(); - } - } - } - public void putCTEIdToConsumer(LogicalCTEConsumer cteConsumer) { Set consumers = this.statementContext.getCteIdToConsumers() .computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 31ecae7f33d6fa..a7bc9dfb18cfd2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -22,13 +22,11 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; -import org.apache.doris.common.FeConstants; import org.apache.doris.common.FormatOptions; import org.apache.doris.common.NereidsException; import org.apache.doris.common.Pair; import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.mysql.FieldInfo; -import org.apache.doris.nereids.CascadesContext.Lock; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.glue.translator.PhysicalPlanTranslator; @@ -52,7 +50,6 @@ import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.ComputeResultSet; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; @@ -143,7 +140,7 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions } @VisibleForTesting - public void planWithLock(StatementBase queryStmt) { + public void plan(StatementBase queryStmt) { try { plan(queryStmt, statementContext.getConnectContext().getSessionVariable().toThrift()); } catch (Exception e) { @@ -151,14 +148,17 @@ public void planWithLock(StatementBase queryStmt) { } } + @VisibleForTesting public PhysicalPlan planWithLock(LogicalPlan plan, PhysicalProperties outputProperties) { return (PhysicalPlan) planWithLock(plan, outputProperties, ExplainLevel.NONE, false); } + // TODO check all caller public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel) { return planWithLock(plan, requireProperties, explainLevel, false); } + @VisibleForTesting public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, boolean showPlanProcess) { Consumer noCallback = p -> {}; @@ -175,9 +175,8 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, * @return plan generated by this planner * @throws AnalysisException throw exception if failed in ant stage */ - public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, - ExplainLevel explainLevel, boolean showPlanProcess, - Consumer lockCallback) { + private Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, + ExplainLevel explainLevel, boolean showPlanProcess, Consumer lockCallback) { try { long beforePlanGcTime = getGarbageCollectionTime(); if (plan instanceof LogicalSqlCache) { @@ -203,39 +202,37 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, plan = preprocess(plan); initCascadesContext(plan, requireProperties); - statementContext.loadSnapshots(cascadesContext.getOrExtractTables(plan)); - try (Lock lock = new Lock(plan, cascadesContext)) { - Plan resultPlan = planWithoutLock(plan, explainLevel, showPlanProcess, requireProperties); - lockCallback.accept(resultPlan); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile() - .setNereidsGarbageCollectionTime(getGarbageCollectionTime() - beforePlanGcTime); - } - return resultPlan; + // collect table and lock them in the order of table id + collectAndLockTable(showAnalyzeProcess(explainLevel, showPlanProcess)); + // after table collector, we should use a new context. + statementContext.loadSnapshots(); + Plan resultPlan = planWithoutLock(plan, requireProperties, explainLevel, showPlanProcess); + lockCallback.accept(resultPlan); + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile() + .setNereidsGarbageCollectionTime(getGarbageCollectionTime() - beforePlanGcTime); } + return resultPlan; } finally { statementContext.releasePlannerResources(); } } + /** + * do plan but not lock any table + */ private Plan planWithoutLock( - LogicalPlan plan, ExplainLevel explainLevel, - boolean showPlanProcess, PhysicalProperties requireProperties) { - // resolve column, table and function - // analyze this query - analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); + LogicalPlan plan, PhysicalProperties requireProperties, ExplainLevel explainLevel, + boolean showPlanProcess) { // minidump of input must be serialized first, this process ensure minidump string not null try { - MinidumpUtils.serializeInputsToDumpFile(plan, cascadesContext.getTables()); + + MinidumpUtils.serializeInputsToDumpFile(plan, statementContext); } catch (IOException e) { throw new RuntimeException(e); } - - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); - } - + // analyze this query, resolve column, table and function + analyze(showAnalyzeProcess(explainLevel, showPlanProcess)); if (explainLevel == ExplainLevel.ANALYZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { analyzedPlan = cascadesContext.getRewritePlan(); if (explainLevel == ExplainLevel.ANALYZED_PLAN) { @@ -245,38 +242,19 @@ private Plan planWithoutLock( // rule-based optimize rewrite(showRewriteProcess(explainLevel, showPlanProcess)); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); - } - if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { rewrittenPlan = cascadesContext.getRewritePlan(); if (explainLevel == ExplainLevel.REWRITTEN_PLAN) { return rewrittenPlan; } } - // if we cannot get table row count, skip join reorder - // except: - // 1. user set leading hint - // 2. ut test. In ut test, FeConstants.enableInternalSchemaDb is false or FeConstants.runningUnitTest is true - if (FeConstants.enableInternalSchemaDb && !FeConstants.runningUnitTest - && !cascadesContext.isLeadingDisableJoinReorder()) { - List scans = cascadesContext.getRewritePlan() - .collectToList(CatalogRelation.class::isInstance); - Optional reason = StatsCalculator.disableJoinReorderIfStatsInvalid(scans, cascadesContext); - reason.ifPresent(LOG::info); - } - setRuntimeFilterWaitTimeByTableRowCountAndType(); - optimize(); - if (statementContext.getConnectContext().getExecutor() != null) { - statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); - } + optimize(); // print memo before choose plan. // if chooseNthPlan failed, we could get memo to debug if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { String memo = cascadesContext.getMemo().toString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + memo); + LOG.info("{}\n{}", ConnectContext.get().getQueryIdentifier(), memo); } int nth = cascadesContext.getConnectContext().getSessionVariable().getNthOptimizedPlan(); @@ -285,7 +263,7 @@ private Plan planWithoutLock( physicalPlan = postProcess(physicalPlan); if (cascadesContext.getConnectContext().getSessionVariable().dumpNereidsMemo) { String tree = physicalPlan.treeString(); - LOG.info(ConnectContext.get().getQueryIdentifier() + "\n" + tree); + LOG.info("{}\n{}", ConnectContext.get().getQueryIdentifier(), tree); } if (explainLevel == ExplainLevel.OPTIMIZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN @@ -344,8 +322,21 @@ private void setRuntimeFilterWaitTimeByTableRowCountAndType() { private void initCascadesContext(LogicalPlan plan, PhysicalProperties requireProperties) { cascadesContext = CascadesContext.initContext(statementContext, plan, requireProperties); - if (statementContext.getConnectContext().getTables() != null) { - cascadesContext.setTables(statementContext.getConnectContext().getTables()); + } + + protected void collectAndLockTable(boolean showPlanProcess) { + if (LOG.isDebugEnabled()) { + LOG.debug("Start collect and lock table"); + } + keepOrShowPlanProcess(showPlanProcess, () -> cascadesContext.newTableCollector().collect()); + statementContext.lock(); + cascadesContext.setCteContext(new CTEContext()); + NereidsTracer.logImportantTime("EndCollectAndLockTables"); + if (LOG.isDebugEnabled()) { + LOG.debug("End collect and lock table"); + } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsLockTableFinishTime(); } } @@ -359,6 +350,11 @@ private void analyze(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("End analyze plan"); } + + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setQueryAnalysisFinishTime(); + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsAnalysisTime(); + } } /** @@ -373,6 +369,9 @@ private void rewrite(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("End rewrite plan"); } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); + } } // DependsRules: EnsureProjectOnTopJoin.class @@ -385,6 +384,9 @@ private void optimize() { if (LOG.isDebugEnabled()) { LOG.debug("End optimize plan"); } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsOptimizeTime(); + } } private void translate(PhysicalPlan resultPlan) { @@ -590,6 +592,8 @@ public String getExplainString(ExplainOptions explainOptions) { plan = "========== PARSED PLAN " + getTimeMetricString(SummaryProfile::getPrettyParseSqlTime) + " ==========\n" + parsedPlan.treeString() + "\n\n" + + "========== LOCK TABLE " + + getTimeMetricString(SummaryProfile::getPrettyNereidsLockTableTime) + " ==========\n" + "========== ANALYZED PLAN " + getTimeMetricString(SummaryProfile::getPrettyNereidsAnalysisTime) + " ==========\n" + analyzedPlan.treeString() + "\n\n" @@ -755,7 +759,7 @@ private boolean showRewriteProcess(ExplainLevel explainLevel, boolean showPlanPr } private boolean showPlanProcess(ExplainOptions explainOptions) { - return explainOptions == null ? false : explainOptions.showPlanProcess(); + return explainOptions != null && explainOptions.showPlanProcess(); } private void keepOrShowPlanProcess(boolean showPlanProcess, Runnable task) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java index 15c87da96ae02b..45a4c1d50d5794 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/SqlCacheContext.java @@ -424,6 +424,10 @@ public static class FullTableName { public String toString() { return catalog + "." + db + "." + table; } + + public List toList() { + return Lists.newArrayList(catalog, db, table); + } } /** FullColumnName */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 3f1fa6614b2edd..5f1c4ea7272101 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; import org.apache.doris.catalog.constraint.TableIdentifier; import org.apache.doris.common.FormatOptions; import org.apache.doris.common.Id; @@ -27,6 +28,7 @@ import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.datasource.mvcc.MvccTable; import org.apache.doris.datasource.mvcc.MvccTableInfo; +import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.rules.analysis.ColumnAliasGenerator; @@ -43,6 +45,7 @@ import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.qe.SessionVariable; @@ -66,11 +69,13 @@ import java.util.ArrayList; import java.util.BitSet; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.PriorityQueue; import java.util.Set; import java.util.Stack; import java.util.TreeMap; @@ -83,6 +88,18 @@ public class StatementContext implements Closeable { private static final Logger LOG = LogManager.getLogger(StatementContext.class); + /** + * indicate where the table come from. + * QUERY: in query sql directly + * INSERT_TARGET: the insert target table + * MTMV: mtmv itself and its related tables witch do not belong to this sql, but maybe used in rewrite by mtmv. + */ + public enum TableFrom { + QUERY, + INSERT_TARGET, + MTMV + } + private ConnectContext connectContext; private final Stopwatch stopwatch = Stopwatch.createUnstarted(); @@ -146,6 +163,18 @@ public class StatementContext implements Closeable { // placeholder params for prepared statement private List placeholders; + // all tables in query + private boolean needLockTables = true; + + // tables in this query directly + private final Map, TableIf> tables = Maps.newHashMap(); + // tables maybe used by mtmv rewritten in this query + private final Map, TableIf> mtmvRelatedTables = Maps.newHashMap(); + // insert into target tables + private final Map, TableIf> insertTargetTables = Maps.newHashMap(); + // save view's def to avoid them change before lock + private final Map, String> viewInfos = Maps.newHashMap(); + // for create view support in nereids // key is the start and end position of the sql substring that needs to be replaced, // and value is the new string used for replacement. @@ -167,7 +196,7 @@ public class StatementContext implements Closeable { private FormatOptions formatOptions = FormatOptions.getDefault(); - private List plannerHooks = new ArrayList<>(); + private final List plannerHooks = new ArrayList<>(); private final Map snapshots = Maps.newHashMap(); @@ -205,6 +234,67 @@ public StatementContext(ConnectContext connectContext, OriginStatement originSta } } + public void setNeedLockTables(boolean needLockTables) { + this.needLockTables = needLockTables; + } + + /** + * cache view info to avoid view's def and sql mode changed before lock it. + * + * @param qualifiedViewName full qualified name of the view + * @param view view need to cache info + * + * @return view info, first is view's def sql, second is view's sql mode + */ + public String getAndCacheViewInfo(List qualifiedViewName, View view) { + return viewInfos.computeIfAbsent(qualifiedViewName, k -> { + String viewDef; + view.readLock(); + try { + viewDef = view.getInlineViewDef(); + } finally { + view.readUnlock(); + } + return viewDef; + }); + } + + public Map, TableIf> getInsertTargetTables() { + return insertTargetTables; + } + + public Map, TableIf> getMtmvRelatedTables() { + return mtmvRelatedTables; + } + + public Map, TableIf> getTables() { + return tables; + } + + public void setTables(Map, TableIf> tables) { + this.tables.clear(); + this.tables.putAll(tables); + } + + /** get table by table name, try to get from information from dumpfile first */ + public TableIf getAndCacheTable(List tableQualifier, TableFrom tableFrom) { + Map, TableIf> tables; + switch (tableFrom) { + case QUERY: + tables = this.tables; + break; + case INSERT_TARGET: + tables = this.insertTargetTables; + break; + case MTMV: + tables = this.mtmvRelatedTables; + break; + default: + throw new AnalysisException("Unknown table from " + tableFrom); + } + return tables.computeIfAbsent(tableQualifier, k -> RelationUtil.getTable(k, connectContext.getEnv())); + } + public void setConnectContext(ConnectContext connectContext) { this.connectContext = connectContext; } @@ -264,10 +354,6 @@ public Optional getSqlCacheContext() { return Optional.ofNullable(sqlCacheContext); } - public void addSlotToRelation(Slot slot, Relation relation) { - slotToRelation.put(slot, relation); - } - public boolean isDpHyp() { return isDpHyp; } @@ -432,21 +518,36 @@ public Map getRelationIdToStatisticsMap() { return relationIdToStatisticsMap; } - /** addTableReadLock */ - public synchronized void addTableReadLock(TableIf tableIf) { - if (!tableIf.needReadLockWhenPlan()) { + /** + * lock all table collect by TableCollector + */ + public synchronized void lock() { + if (!needLockTables + || (tables.isEmpty() && mtmvRelatedTables.isEmpty() && insertTargetTables.isEmpty()) + || !plannerResources.isEmpty()) { return; } - if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { - close(); - throw new RuntimeException(String.format("Failed to get read lock on table: %s", tableIf.getName())); + PriorityQueue tableIfs = new PriorityQueue<>( + tables.size() + mtmvRelatedTables.size() + insertTargetTables.size(), + Comparator.comparing(TableIf::getId)); + tableIfs.addAll(tables.values()); + tableIfs.addAll(mtmvRelatedTables.values()); + tableIfs.addAll(insertTargetTables.values()); + while (!tableIfs.isEmpty()) { + TableIf tableIf = tableIfs.poll(); + if (!tableIf.needReadLockWhenPlan()) { + continue; + } + if (!tableIf.tryReadLock(1, TimeUnit.MINUTES)) { + close(); + throw new RuntimeException("Failed to get read lock on table:" + tableIf.getName()); + } + String fullTableName = tableIf.getNameWithFullQualifiers(); + String resourceName = "tableReadLock(" + fullTableName + ")"; + plannerResources.push(new CloseableResource( + resourceName, Thread.currentThread().getName(), + originStatement == null ? null : originStatement.originStmt, tableIf::readUnlock)); } - - String fullTableName = tableIf.getNameWithFullQualifiers(); - String resourceName = "tableReadLock(" + fullTableName + ")"; - plannerResources.push(new CloseableResource( - resourceName, Thread.currentThread().getName(), - originStatement == null ? null : originStatement.originStmt, tableIf::readUnlock)); } /** releasePlannerResources */ @@ -509,13 +610,9 @@ public void addPlannerHook(PlannerHook plannerHook) { /** * Load snapshot information of mvcc - * - * @param tables Tables used in queries */ - public void loadSnapshots(Map tables) { - if (tables == null) { - return; - } + + public void loadSnapshots() { for (TableIf tableIf : tables.values()) { if (tableIf instanceof MvccTable) { snapshots.put(new MvccTableInfo(tableIf), ((MvccTable) tableIf).loadSnapshot()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java index cfdefc59872d4e..2c88e2f4a46ff7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundBaseExternalTableSink.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -80,11 +79,6 @@ public UnboundBaseExternalTableSink withOutputExprs(List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java index 9bc368c8ad6b18..bb61bc93574208 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java @@ -71,7 +71,7 @@ public List getProjects() { @Override public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); + return projects; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index b8d821e1548be7..12d4a7c74be58f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -25,7 +25,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; import org.apache.doris.nereids.trees.TableSample; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; import org.apache.doris.nereids.trees.plans.Plan; @@ -186,11 +185,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundRelation(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - public List getPartNames() { return partNames; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java index 10a6b0121e3b27..7e86a45017cf06 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java @@ -20,7 +20,6 @@ import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -63,11 +62,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundResultSink(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public Plan withGroupExpression(Optional groupExpression) { return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), child()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java index e876825af6569a..3024058edc7a5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Properties; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; @@ -79,11 +78,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundTVFRelation(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public List computeOutput() { throw new UnboundException("output"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java index 23c58ba42fb17e..0e528227dc9742 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTableSink.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.BlockFuncDepsPropagation; @@ -135,11 +134,6 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundTableSink(this, context); } - @Override - public List getExpressions() { - throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 1ffbac97d741a4..3a111a7f4d776f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -24,7 +24,6 @@ import org.apache.doris.nereids.rules.analysis.AnalyzeCTE; import org.apache.doris.nereids.rules.analysis.BindExpression; import org.apache.doris.nereids.rules.analysis.BindRelation; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.analysis.BindSink; import org.apache.doris.nereids.rules.analysis.CheckAfterBind; import org.apache.doris.nereids.rules.analysis.CheckAnalysis; @@ -54,8 +53,6 @@ import com.google.common.collect.ImmutableSet; import java.util.List; -import java.util.Objects; -import java.util.Optional; /** * Bind symbols according to metadata in the catalog, perform semantic analysis, etc. @@ -63,38 +60,20 @@ */ public class Analyzer extends AbstractBatchJobExecutor { - public static final List ANALYZE_JOBS = buildAnalyzeJobs(Optional.empty()); - - private final List jobs; - - /** - * Execute the analysis job with scope. - * @param cascadesContext planner context for execute job - */ - public Analyzer(CascadesContext cascadesContext) { - this(cascadesContext, Optional.empty()); - } + public static final List ANALYZE_JOBS = buildAnalyzeJobs(); /** * constructor of Analyzer. For view, we only do bind relation since other analyze step will do by outer Analyzer. * * @param cascadesContext current context for analyzer - * @param customTableResolver custom resolver for outer catalog. */ - public Analyzer(CascadesContext cascadesContext, Optional customTableResolver) { + public Analyzer(CascadesContext cascadesContext) { super(cascadesContext); - Objects.requireNonNull(customTableResolver, "customTableResolver cannot be null"); - - if (customTableResolver.isPresent()) { - this.jobs = buildAnalyzeJobs(customTableResolver); - } else { - this.jobs = ANALYZE_JOBS; - } } @Override public List getJobs() { - return jobs; + return ANALYZE_JOBS; } /** @@ -104,20 +83,20 @@ public void analyze() { execute(); } - private static List buildAnalyzeJobs(Optional customTableResolver) { + private static List buildAnalyzeJobs() { return notTraverseChildrenOf( ImmutableSet.of(LogicalView.class, LogicalCTEAnchor.class), - () -> buildAnalyzerJobs(customTableResolver) + Analyzer::buildAnalyzerJobs ); } - private static List buildAnalyzerJobs(Optional customTableResolver) { + private static List buildAnalyzerJobs() { return jobs( // we should eliminate hint before "Subquery unnesting". topDown(new AnalyzeCTE()), topDown(new EliminateLogicalSelectHint()), bottomUp( - new BindRelation(customTableResolver), + new BindRelation(), new CheckPolicy() ), bottomUp(new BindExpression()), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java new file mode 100644 index 00000000000000..0ae433262efeb9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.jobs.executor; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.rules.analysis.CollectRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalView; + +import com.google.common.collect.ImmutableSet; + +import java.util.List; + +/** + * Bind symbols according to metadata in the catalog, perform semantic analysis, etc. + * TODO: revisit the interface after subquery analysis is supported. + */ +public class TableCollector extends AbstractBatchJobExecutor { + + public static final List COLLECT_JOBS = buildCollectTableJobs(); + + /** + * constructor of Analyzer. For view, we only do bind relation since other analyze step will do by outer Analyzer. + * + * @param cascadesContext current context for analyzer + */ + public TableCollector(CascadesContext cascadesContext) { + super(cascadesContext); + + } + + @Override + public List getJobs() { + return COLLECT_JOBS; + } + + /** + * nereids analyze sql. + */ + public void collect() { + execute(); + } + + private static List buildCollectTableJobs() { + return notTraverseChildrenOf( + ImmutableSet.of(LogicalView.class), + TableCollector::buildCollectorJobs + ); + } + + private static List buildCollectorJobs() { + return jobs( + topDown(new CollectRelation()) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java index 8d6d9b0e7946ef..334e5b5c3cc69e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/minidump/MinidumpUtils.java @@ -41,6 +41,8 @@ import org.apache.doris.statistics.ColumnStatisticBuilder; import org.apache.doris.statistics.Histogram; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.json.JSONArray; @@ -214,7 +216,7 @@ public static JSONObject executeSql(String sql) { } NereidsPlanner nereidsPlanner = new NereidsPlanner( new StatementContext(ConnectContext.get(), new OriginStatement(sql, 0))); - nereidsPlanner.planWithLock(LogicalPlanAdapter.of(parsed)); + nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); return ((AbstractPlan) nereidsPlanner.getOptimizedPlan()).toJson(); } @@ -460,10 +462,11 @@ private static JSONObject serializeInputs(Plan parsedPlan, List tables) /** * This function is used to serialize inputs of one query * @param parsedPlan input plan - * @param tables all tables relative to this query + * @param statementContext context for this query * @throws IOException this will write to disk, so io exception should be dealed with */ - public static void serializeInputsToDumpFile(Plan parsedPlan, List tables) throws IOException { + public static void serializeInputsToDumpFile(Plan parsedPlan, StatementContext statementContext) + throws IOException { ConnectContext connectContext = ConnectContext.get(); // when playing minidump file, we do not save input again. if (connectContext.getSessionVariable().isPlayNereidsDump() @@ -477,7 +480,10 @@ public static void serializeInputsToDumpFile(Plan parsedPlan, List tabl connectContext.getSessionVariable().setMinidumpPath("defaultMinidumpPath"); } MinidumpUtils.init(); - connectContext.setMinidump(serializeInputs(parsedPlan, tables)); + Map, TableIf> allTablesUsedInQuery = Maps.newHashMap(); + allTablesUsedInQuery.putAll(statementContext.getTables()); + allTablesUsedInQuery.putAll(statementContext.getMtmvRelatedTables()); + connectContext.setMinidump(serializeInputs(parsedPlan, Lists.newArrayList(allTablesUsedInQuery.values()))); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ad20280a6d81dc..50e15f8791c3d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -620,7 +620,8 @@ public LogicalPlan visitInsertTable(InsertTableContext ctx) { command = new InsertOverwriteTableCommand(sink, labelName, cte); } else { if (ConnectContext.get() != null && ConnectContext.get().isTxnModel() - && sink.child() instanceof LogicalInlineTable) { + && sink.child() instanceof LogicalInlineTable + && sink.child().getExpressions().stream().allMatch(Expression::isConstant)) { // FIXME: In legacy, the `insert into select 1` is handled as `insert into values`. // In nereids, the original way is throw an AnalysisException and fallback to legacy. // Now handle it as `insert into select`(a separate load job), should fix it as the legacy. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index f6445ba5878b9d..f0296f6f80f2da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -26,9 +26,14 @@ public enum RuleType { // just for UT TEST_REWRITE(RuleTypeClass.REWRITE), - // binding rules - // **** make sure BINDING_UNBOUND_LOGICAL_PLAN is the lowest priority in the rewrite rules. **** + // collect relation rules + COLLECT_TABLE_FROM_CTE(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_RELATION(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_SINK(RuleTypeClass.REWRITE), + COLLECT_TABLE_FROM_OTHER(RuleTypeClass.REWRITE), + + // binding rules BINDING_RESULT_SINK(RuleTypeClass.REWRITE), BINDING_INSERT_HIVE_TABLE(RuleTypeClass.REWRITE), BINDING_INSERT_ICEBERG_TABLE(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 36363650cd099e..6d93c8d5218530 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.SqlCacheContext; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.StatementContext.TableFrom; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -98,25 +99,14 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.function.Function; /** * Rule to bind relations in query plan. */ public class BindRelation extends OneAnalysisRuleFactory { - private final Optional customTableResolver; + public BindRelation() {} - public BindRelation() { - this(Optional.empty()); - } - - public BindRelation(Optional customTableResolver) { - this.customTableResolver = customTableResolver; - } - - // TODO: cte will be copied to a sub-query with different names but the id of the unbound relation in them - // are the same, so we use new relation id when binding relation, and will fix this bug later. @Override public Rule build() { return unboundRelation().thenApply(ctx -> { @@ -168,19 +158,11 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe return consumer; } } - List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), - unboundRelation.getNameParts()); - TableIf table = null; - if (customTableResolver.isPresent()) { - table = customTableResolver.get().apply(tableQualifier); - } - // In some cases even if we have already called the "cascadesContext.getTableByName", - // it also gets the null. So, we just check it in the catalog again for safety. - if (table == null) { - table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); - } - // TODO: should generate different Scan sub class according to table's type + List tableQualifier = RelationUtil.getQualifierName( + cascadesContext.getConnectContext(), unboundRelation.getNameParts()); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); + LogicalPlan scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); if (cascadesContext.isLeadingJoin()) { LeadingHint leading = (LeadingHint) cascadesContext.getHintMap().get("Leading"); @@ -191,18 +173,10 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe } private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboundRelation) { - List qualifiedTablName = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), + List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), unboundRelation.getNameParts()); - TableIf table = null; - if (customTableResolver.isPresent()) { - table = customTableResolver.get().apply(qualifiedTablName); - } - // In some cases even if we have already called the "cascadesContext.getTableByName", - // it also gets the null. So, we just check it in the catalog again for safety. - if (table == null) { - table = RelationUtil.getTable(qualifiedTablName, cascadesContext.getConnectContext().getEnv()); - } - return getLogicalPlan(table, unboundRelation, qualifiedTablName, cascadesContext); + TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY); + return getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); } private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, List qualifier) { @@ -409,8 +383,7 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio case VIEW: View view = (View) table; isView = true; - String inlineViewDef = view.getInlineViewDef(); - Plan viewBody = parseAndAnalyzeView(view, inlineViewDef, cascadesContext); + Plan viewBody = parseAndAnalyzeDorisView(view, qualifiedTableName, cascadesContext); LogicalView logicalView = new LogicalView<>(view, viewBody); return new LogicalSubQueryAlias<>(qualifiedTableName, logicalView); case HMS_EXTERNAL_TABLE: @@ -488,6 +461,11 @@ private Plan parseAndAnalyzeHiveView( } } + private Plan parseAndAnalyzeDorisView(View view, List tableQualifier, CascadesContext parentContext) { + String viewDef = parentContext.getStatementContext().getAndCacheViewInfo(tableQualifier, view); + return parseAndAnalyzeView(view, viewDef, parentContext); + } + private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext parentContext) { parentContext.getStatementContext().addViewDdlSql(ddlSql); Optional sqlCacheContext = parentContext.getStatementContext().getSqlCacheContext(); @@ -502,7 +480,7 @@ private Plan parseAndAnalyzeView(TableIf view, String ddlSql, CascadesContext pa CascadesContext viewContext = CascadesContext.initContext( parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); viewContext.keepOrShowPlanProcess(parentContext.showPlanProcess(), () -> { - viewContext.newAnalyzer(customTableResolver).analyze(); + viewContext.newAnalyzer().analyze(); }); parentContext.addPlanProcesses(viewContext.getPlanProcesses()); // we should remove all group expression of the plan which in other memo, so the groupId would not conflict @@ -535,7 +513,4 @@ private List getPartitionIds(TableIf t, UnboundRelation unboundRelation, L return part.getId(); }).collect(ImmutableList.toImmutableList()); } - - /** CustomTableResolver */ - public interface CustomTableResolver extends Function, TableIf> {} } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java new file mode 100644 index 00000000000000..bf8c53259ce6ad --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -0,0 +1,220 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.analysis; + +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.View; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.CTEContext; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext.TableFrom; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.analyzer.UnboundTableSink; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.pattern.MatchingContext; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.SubqueryExpr; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; +import org.apache.doris.nereids.util.RelationUtil; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Rule to bind relations in query plan. + */ +public class CollectRelation implements AnalysisRuleFactory { + + private static final Logger LOG = LogManager.getLogger(CollectRelation.class); + + public CollectRelation() {} + + @Override + public List buildRules() { + return ImmutableList.of( + // should collect table from cte first to fill collect all cte name to avoid collect wrong table. + logicalCTE() + .thenApply(ctx -> { + ctx.cascadesContext.setCteContext(collectFromCte(ctx.root, ctx.cascadesContext)); + return null; + }) + .toRule(RuleType.COLLECT_TABLE_FROM_CTE), + unboundRelation() + .thenApply(this::collectFromUnboundRelation) + .toRule(RuleType.COLLECT_TABLE_FROM_RELATION), + unboundTableSink() + .thenApply(this::collectFromUnboundTableSink) + .toRule(RuleType.COLLECT_TABLE_FROM_SINK), + any().whenNot(UnboundRelation.class::isInstance) + .whenNot(UnboundTableSink.class::isInstance) + .thenApply(this::collectFromAny) + .toRule(RuleType.COLLECT_TABLE_FROM_OTHER) + ); + } + + /** + * register and store CTEs in CTEContext + */ + private CTEContext collectFromCte( + LogicalCTE logicalCTE, CascadesContext cascadesContext) { + CTEContext outerCteCtx = cascadesContext.getCteContext(); + List> aliasQueries = logicalCTE.getAliasQueries(); + for (LogicalSubQueryAlias aliasQuery : aliasQueries) { + // we should use a chain to ensure visible of cte + LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); + CascadesContext innerCascadesCtx = CascadesContext.newContextWithCteContext( + cascadesContext, parsedCtePlan, outerCteCtx); + innerCascadesCtx.newTableCollector().collect(); + LogicalPlan analyzedCtePlan = (LogicalPlan) innerCascadesCtx.getRewritePlan(); + // cteId is not used in CollectTable stage + CTEId cteId = new CTEId(0); + LogicalSubQueryAlias logicalSubQueryAlias = + aliasQuery.withChildren(ImmutableList.of(analyzedCtePlan)); + outerCteCtx = new CTEContext(cteId, logicalSubQueryAlias, outerCteCtx); + outerCteCtx.setAnalyzedPlan(logicalSubQueryAlias); + } + return outerCteCtx; + } + + private Plan collectFromAny(MatchingContext ctx) { + Set subqueryExprs = ctx.root.getExpressions().stream() + .>map(p -> p.collect(SubqueryExpr.class::isInstance)) + .flatMap(Set::stream) + .collect(Collectors.toSet()); + for (SubqueryExpr subqueryExpr : subqueryExprs) { + CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( + ctx.cascadesContext, subqueryExpr.getQueryPlan(), ctx.cteContext); + subqueryContext.keepOrShowPlanProcess(ctx.cascadesContext.showPlanProcess(), + () -> subqueryContext.newTableCollector().collect()); + ctx.cascadesContext.addPlanProcesses(subqueryContext.getPlanProcesses()); + } + return null; + } + + private Plan collectFromUnboundTableSink(MatchingContext> ctx) { + List nameParts = ctx.root.getNameParts(); + switch (nameParts.size()) { + case 1: + // table + // Use current database name from catalog. + case 2: + // db.table + // Use database name from table name parts. + case 3: + // catalog.db.table + // Use catalog and database name from name parts. + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.INSERT_TARGET); + return null; + default: + throw new IllegalStateException("Insert target name is invalid."); + } + } + + private Plan collectFromUnboundRelation(MatchingContext ctx) { + List nameParts = ctx.root.getNameParts(); + switch (nameParts.size()) { + case 1: + // table + // Use current database name from catalog. + case 2: + // db.table + // Use database name from table name parts. + case 3: + // catalog.db.table + // Use catalog and database name from name parts. + collectFromUnboundRelation(ctx.cascadesContext, nameParts, TableFrom.QUERY); + return null; + default: + throw new IllegalStateException("Table name [" + ctx.root.getTableName() + "] is invalid."); + } + } + + private void collectFromUnboundRelation(CascadesContext cascadesContext, + List nameParts, TableFrom tableFrom) { + if (nameParts.size() == 1) { + String tableName = nameParts.get(0); + // check if it is a CTE's name + CTEContext cteContext = cascadesContext.getCteContext().findCTEContext(tableName).orElse(null); + if (cteContext != null) { + Optional analyzedCte = cteContext.getAnalyzedCTEPlan(tableName); + if (analyzedCte.isPresent()) { + return; + } + } + } + List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), nameParts); + TableIf table = cascadesContext.getConnectContext().getStatementContext() + .getAndCacheTable(tableQualifier, tableFrom); + LOG.info("collect table {} from {}", nameParts, tableFrom); + if (tableFrom == TableFrom.QUERY) { + collectMTMVCandidates(table, cascadesContext); + } + if (table instanceof View) { + parseAndCollectFromView(tableQualifier, (View) table, cascadesContext); + } + } + + private void collectMTMVCandidates(TableIf table, CascadesContext cascadesContext) { + if (cascadesContext.getConnectContext().getSessionVariable().enableMaterializedViewRewrite) { + Set mtmvSet = Env.getCurrentEnv().getMtmvService().getRelationManager() + .getAllMTMVs(Lists.newArrayList(new BaseTableInfo(table))); + LOG.info("table {} related mv set is {}", new BaseTableInfo(table), mtmvSet); + for (MTMV mtmv : mtmvSet) { + cascadesContext.getStatementContext().getMtmvRelatedTables().put(mtmv.getFullQualifiers(), mtmv); + mtmv.readMvLock(); + try { + for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTables()) { + LOG.info("mtmv {} related base table include {}", new BaseTableInfo(mtmv), baseTableInfo); + cascadesContext.getStatementContext().getAndCacheTable(baseTableInfo.toList(), TableFrom.MTMV); + } + } finally { + mtmv.readMvUnlock(); + } + } + } + } + + private void parseAndCollectFromView(List tableQualifier, View view, CascadesContext parentContext) { + String viewDef = parentContext.getStatementContext().getAndCacheViewInfo(tableQualifier, view); + LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(viewDef); + if (parsedViewPlan instanceof UnboundResultSink) { + parsedViewPlan = (LogicalPlan) ((UnboundResultSink) parsedViewPlan).child(); + } + CascadesContext viewContext = CascadesContext.initContext( + parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); + viewContext.keepOrShowPlanProcess(parentContext.showPlanProcess(), + () -> viewContext.newTableCollector().collect()); + parentContext.addPlanProcesses(viewContext.getPlanProcesses()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java index 96d37ad546a7b4..593ad986ca797c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java @@ -131,11 +131,6 @@ boolean isFinalChosen(Relation relation) { @Override public Plan getScanPlan(StructInfo queryInfo, CascadesContext cascadesContext) { - // If try to get scan plan or rewrite successfully, try to get mv read lock to avoid meta data inconsistent, - // try to get lock which should added before RBO - if (!this.isSuccess()) { - cascadesContext.getStatementContext().addTableReadLock(this.getMtmv()); - } super.getScanPlan(queryInfo, cascadesContext); return scanPlan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index 925e8dba3c31fa..76651aa1169da8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -26,9 +26,6 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.PlannerHook; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; @@ -70,30 +67,17 @@ public void initMaterializationContext(CascadesContext cascadesContext) { */ protected void doInitMaterializationContext(CascadesContext cascadesContext) { if (cascadesContext.getConnectContext().getSessionVariable().isInDebugMode()) { - LOG.info(String.format("MaterializationContext init return because is in debug mode, current queryId is %s", - cascadesContext.getConnectContext().getQueryIdentifier())); + LOG.info("MaterializationContext init return because is in debug mode, current queryId is {}", + cascadesContext.getConnectContext().getQueryIdentifier()); return; } - // Only collect the table or mv which query use directly, to avoid useless mv partition in rewrite - // Keep use one connection context when in query, if new connect context, - // the ConnectionContext.get() will change - TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet(), false, - cascadesContext.getConnectContext()); - try { - Plan rewritePlan = cascadesContext.getRewritePlan(); - rewritePlan.accept(TableCollector.INSTANCE, collectorContext); - } catch (Exception e) { - LOG.warn(String.format("MaterializationContext init table collect fail, current queryId is %s", - cascadesContext.getConnectContext().getQueryIdentifier()), e); - return; - } - Set collectedTables = collectorContext.getCollectedTables(); + Set collectedTables = Sets.newHashSet(cascadesContext.getStatementContext().getTables().values()); if (collectedTables.isEmpty()) { return; } // Create async materialization context for (MaterializationContext context : createAsyncMaterializationContext(cascadesContext, - collectorContext.getCollectedTables())) { + collectedTables)) { cascadesContext.addMaterializationContext(context); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index f87f67d2e7a77c..2a37ee6219da07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -127,11 +127,10 @@ public static RelatedTableInfo getRelatedTableInfo(String column, String timeUni materializedViewPlan = new LogicalProject<>(ImmutableList.of(columnExpr), materializedViewPlan); } // Collect table relation map which is used to identify self join - List catalogRelationObjs = - materializedViewPlan.collectToList(CatalogRelation.class::isInstance); + List catalogRelations = materializedViewPlan.collectToList(CatalogRelation.class::isInstance); ImmutableMultimap.Builder tableCatalogRelationMultimapBuilder = ImmutableMultimap.builder(); - for (CatalogRelation catalogRelation : catalogRelationObjs) { + for (CatalogRelation catalogRelation : catalogRelations) { tableCatalogRelationMultimapBuilder.put(new TableIdentifier(catalogRelation.getTable()), catalogRelation); } // Check sql pattern @@ -319,6 +318,7 @@ public static MTMVCache createMTMVCache(String querySql, ConnectContext connectC LogicalPlan unboundMvPlan = new NereidsParser().parseSingle(querySql); StatementContext mvSqlStatementContext = new StatementContext(connectContext, new OriginStatement(querySql, 0)); + mvSqlStatementContext.setNeedLockTables(false); NereidsPlanner planner = new NereidsPlanner(mvSqlStatementContext); if (mvSqlStatementContext.getConnectContext().getStatementContext() == null) { mvSqlStatementContext.getConnectContext().setStatementContext(mvSqlStatementContext); @@ -769,7 +769,7 @@ public static final class RelatedTableInfo { private final String column; private final Set failReasons = new HashSet<>(); // This records the partition expression if exist - private Optional partitionExpression; + private final Optional partitionExpression; public RelatedTableInfo(BaseTableInfo tableInfo, boolean pctPossible, String column, String failReason, Expression partitionExpression) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java index 08954741c806aa..f92a3b6103b345 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AddConstraintCommand.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -34,9 +35,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Comparator; +import java.util.List; import java.util.Set; /** @@ -61,15 +65,26 @@ public AddConstraintCommand(String name, Constraint constraint) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { Pair, TableIf> columnsAndTable = extractColumnsAndTable(ctx, constraint.toProject()); + List tables = Lists.newArrayList(columnsAndTable.second); + Pair, TableIf> referencedColumnsAndTable = null; if (constraint.isForeignKey()) { - Pair, TableIf> referencedColumnsAndTable - = extractColumnsAndTable(ctx, constraint.toReferenceProject()); - columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, - referencedColumnsAndTable.second, referencedColumnsAndTable.first, false); - } else if (constraint.isPrimaryKey()) { - columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first, false); - } else if (constraint.isUnique()) { - columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first, false); + referencedColumnsAndTable = extractColumnsAndTable(ctx, constraint.toReferenceProject()); + tables.add(referencedColumnsAndTable.second); + } + tables.sort((Comparator.comparing(TableIf::getId))); + MetaLockUtils.writeLockTables(tables); + try { + if (constraint.isForeignKey()) { + Preconditions.checkState(referencedColumnsAndTable != null); + columnsAndTable.second.addForeignConstraint(name, columnsAndTable.first, + referencedColumnsAndTable.second, referencedColumnsAndTable.first, false); + } else if (constraint.isPrimaryKey()) { + columnsAndTable.second.addPrimaryKeyConstraint(name, columnsAndTable.first, false); + } else if (constraint.isUnique()) { + columnsAndTable.second.addUniqueConstraint(name, columnsAndTable.first, false); + } + } finally { + MetaLockUtils.writeUnlockTables(tables); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java deleted file mode 100644 index f9b0c3e18d1b2f..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java +++ /dev/null @@ -1,49 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.plans.commands; - -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.util.RelationUtil; -import org.apache.doris.qe.ConnectContext; - -import java.util.List; - -/** - * delete from unique key table. - */ -public class CommandUtils { - - /** - * check delete target table should unique key olap table. If ok, return it. - */ - public static OlapTable checkAndGetDeleteTargetTable(ConnectContext ctx, List nameParts) { - List qualifiedTableName = RelationUtil.getQualifierName(ctx, nameParts); - TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv()); - if (!(table instanceof OlapTable)) { - throw new AnalysisException("table must be olapTable in delete command"); - } - OlapTable targetTable = ((OlapTable) table); - if (targetTable.getKeysType() != KeysType.UNIQUE_KEYS) { - throw new AnalysisException("Nereids only support delete command on unique key table now"); - } - return targetTable; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java index fe777ea82972fc..63d432121599e0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DropConstraintCommand.java @@ -18,6 +18,9 @@ package org.apache.doris.nereids.trees.plans.commands; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.constraint.Constraint; +import org.apache.doris.catalog.constraint.PrimaryKeyConstraint; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -30,9 +33,12 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; +import com.google.common.collect.Lists; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Comparator; +import java.util.List; import java.util.Set; /** @@ -56,7 +62,27 @@ public DropConstraintCommand(String name, LogicalPlan plan) { @Override public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf table = extractTable(ctx, plan); - table.dropConstraint(name, false); + List tables = Lists.newArrayList(table); + table.readLock(); + try { + Constraint constraint = table.getConstraintsMapUnsafe().get(name); + if (constraint == null) { + throw new AnalysisException( + String.format("Unknown constraint %s on table %s.", name, table.getName())); + } + if (constraint instanceof PrimaryKeyConstraint) { + tables.addAll(((PrimaryKeyConstraint) constraint).getForeignTables()); + } + } finally { + table.readUnlock(); + } + tables.sort((Comparator.comparing(TableIf::getId))); + MetaLockUtils.writeLockTables(tables); + try { + table.dropConstraint(name, false); + } finally { + MetaLockUtils.writeUnlockTables(tables); + } } private TableIf extractTable(ConnectContext ctx, LogicalPlan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java index 13d1f0c5f6d2a2..534a365e07716c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowConstraintsCommand.java @@ -51,12 +51,19 @@ public ShowConstraintsCommand(List nameParts) { public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { TableIf tableIf = RelationUtil.getDbAndTable( RelationUtil.getQualifierName(ctx, nameParts), ctx.getEnv()).value(); - List> res = tableIf.getConstraintsMap().entrySet().stream() - .map(e -> Lists.newArrayList(e.getKey(), - e.getValue().getType().getName(), - e.getValue().toString())) + tableIf.readLock(); + List> res; + try { + res = tableIf.getConstraintsMap().entrySet().stream() + .map(e -> Lists.newArrayList(e.getKey(), + e.getValue().getType().getName(), + e.getValue().toString())) .collect(Collectors.toList()); + } finally { + tableIf.readUnlock(); + } executor.handleShowConstraintStmt(res); + } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 867b22dcde746e..c4b6286d19382a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -249,42 +249,43 @@ private void analyzeProperties() { /** * analyzeQuery */ - public void analyzeQuery(ConnectContext ctx, Map mvProperties) throws Exception { - // create table as select - StatementContext statementContext = ctx.getStatementContext(); - NereidsPlanner planner = new NereidsPlanner(statementContext); - // this is for expression column name infer when not use alias - LogicalSink logicalSink = new UnboundResultSink<>(logicalQuery); - // Should not make table without data to empty relation when analyze the related table, - // so add disable rules - Set tempDisableRules = ctx.getSessionVariable().getDisableNereidsRuleNames(); - ctx.getSessionVariable().setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - Plan plan; - try { - // must disable constant folding by be, because be constant folding may return wrong type - ctx.getSessionVariable().disableConstantFoldingByBEOnce(); - plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); - } finally { - // after operate, roll back the disable rules - ctx.getSessionVariable().setDisableNereidsRules(String.join(",", tempDisableRules)); - ctx.getStatementContext().invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); - } - // can not contain VIEW or MTMV - analyzeBaseTables(planner.getAnalyzedPlan()); - // can not contain Random function - analyzeExpressions(planner.getAnalyzedPlan(), mvProperties); - // can not contain partition or tablets - boolean containTableQueryOperator = MaterializedViewUtils.containTableQueryOperator(planner.getAnalyzedPlan()); - if (containTableQueryOperator) { - throw new AnalysisException("can not contain invalid expression"); - } - getRelation(planner); - this.mvPartitionInfo = mvPartitionDefinition.analyzeAndTransferToMTMVPartitionInfo(planner, ctx); - this.partitionDesc = generatePartitionDesc(ctx); - getColumns(plan, ctx, mvPartitionInfo.getPartitionCol(), distribution); - analyzeKeys(); + public void analyzeQuery(ConnectContext ctx, Map mvProperties) { + try (StatementContext statementContext = ctx.getStatementContext()) { + NereidsPlanner planner = new NereidsPlanner(statementContext); + // this is for expression column name infer when not use alias + LogicalSink logicalSink = new UnboundResultSink<>(logicalQuery); + // Should not make table without data to empty relation when analyze the related table, + // so add disable rules + Set tempDisableRules = ctx.getSessionVariable().getDisableNereidsRuleNames(); + ctx.getSessionVariable().setDisableNereidsRules(CreateMTMVInfo.MTMV_PLANER_DISABLE_RULES); + statementContext.invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + Plan plan; + try { + // must disable constant folding by be, because be constant folding may return wrong type + ctx.getSessionVariable().setVarOnce(SessionVariable.ENABLE_FOLD_CONSTANT_BY_BE, "false"); + plan = planner.planWithLock(logicalSink, PhysicalProperties.ANY, ExplainLevel.ALL_PLAN); + } finally { + // after operate, roll back the disable rules + ctx.getSessionVariable().setDisableNereidsRules(String.join(",", tempDisableRules)); + statementContext.invalidCache(SessionVariable.DISABLE_NEREIDS_RULES); + } + // can not contain VIEW or MTMV + analyzeBaseTables(planner.getAnalyzedPlan()); + // can not contain Random function + analyzeExpressions(planner.getAnalyzedPlan(), mvProperties); + // can not contain partition or tablets + boolean containTableQueryOperator = MaterializedViewUtils.containTableQueryOperator( + planner.getAnalyzedPlan()); + if (containTableQueryOperator) { + throw new AnalysisException("can not contain invalid expression"); + } + getRelation(Sets.newHashSet(statementContext.getTables().values()), ctx); + this.mvPartitionInfo = mvPartitionDefinition.analyzeAndTransferToMTMVPartitionInfo(planner); + this.partitionDesc = generatePartitionDesc(ctx); + getColumns(plan, ctx, mvPartitionInfo.getPartitionCol(), distribution); + analyzeKeys(); + } } private void analyzeKeys() { @@ -326,9 +327,9 @@ private void analyzeKeys() { } // Should use analyzed plan for collect views and tables - private void getRelation(NereidsPlanner planner) { - this.relation = MTMVPlanUtil.generateMTMVRelation(planner.getAnalyzedPlan(), - planner.getCascadesContext().getConnectContext()); + + private void getRelation(Set tables, ConnectContext ctx) { + this.relation = MTMVPlanUtil.generateMTMVRelation(tables, ctx); } private PartitionDesc generatePartitionDesc(ConnectContext ctx) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index a26a97f7240793..c2e9abd2f0f97c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -44,7 +44,6 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Sets; @@ -66,10 +65,9 @@ public class MTMVPartitionDefinition { * analyzeAndTransferToMTMVPartitionInfo * * @param planner planner - * @param ctx ctx * @return MTMVPartitionInfo */ - public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner, ConnectContext ctx) { + public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner planner) { MTMVPartitionInfo mtmvPartitionInfo = new MTMVPartitionInfo(partitionType); if (this.partitionType == MTMVPartitionType.SELF_MANAGE) { return mtmvPartitionInfo; @@ -77,9 +75,8 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl String partitionColName; String timeUnit; if (this.partitionType == MTMVPartitionType.EXPR) { - String functionName = ((UnboundFunction) functionCallExpression).getName(); - if (functionCallExpression instanceof UnboundFunction - && functionName.equalsIgnoreCase(PARTITION_BY_FUNCTION_NAME)) { + if (functionCallExpression instanceof UnboundFunction && PARTITION_BY_FUNCTION_NAME + .equalsIgnoreCase(((UnboundFunction) functionCallExpression).getName())) { partitionColName = functionCallExpression.getArgument(0) instanceof UnboundSlot ? ((UnboundSlot) functionCallExpression.getArgument(0)).getName() : null; timeUnit = functionCallExpression.getArguments().get(1).isLiteral() @@ -93,7 +90,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl timeUnit = null; } mtmvPartitionInfo.setPartitionCol(partitionColName); - RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, ctx, partitionColName, timeUnit); + RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, partitionColName, timeUnit); mtmvPartitionInfo.setRelatedCol(relatedTableInfo.getColumn()); mtmvPartitionInfo.setRelatedTable(relatedTableInfo.getTableInfo()); if (relatedTableInfo.getPartitionExpression().isPresent()) { @@ -119,8 +116,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl } // Should use rewritten plan without view and subQuery to get related partition table - private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectContext ctx, - String partitionColName, String timeUnit) { + private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, String partitionColName, String timeUnit) { CascadesContext cascadesContext = planner.getCascadesContext(); RelatedTableInfo relatedTableInfo = MaterializedViewUtils @@ -129,10 +125,10 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectCont throw new AnalysisException(String.format("Unable to find a suitable base table for partitioning," + " the fail reason is %s", relatedTableInfo.getFailReason())); } - MTMVRelatedTableIf mtmvBaseRealtedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); + MTMVRelatedTableIf mtmvBaseRelatedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); try { - partitionColumnNames.addAll(mtmvBaseRealtedTable.getPartitionColumnNames(Optional.empty())); + partitionColumnNames.addAll(mtmvBaseRelatedTable.getPartitionColumnNames(Optional.empty())); } catch (DdlException e) { throw new AnalysisException(e.getMessage(), e); } @@ -140,7 +136,7 @@ private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, ConnectCont if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); } - if (!(mtmvBaseRealtedTable instanceof HMSExternalTable) + if (!(mtmvBaseRelatedTable instanceof HMSExternalTable) && partitionColumnNames.size() != 1) { throw new AnalysisException("only hms table support multi column partition."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 59b99c209d279c..f054d57c88616f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.ProfileManager.ProfileType; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalTable; @@ -49,12 +50,14 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.planner.DataSink; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectContext.ConnectType; import org.apache.doris.qe.StmtExecutor; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -74,13 +77,14 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, public static final Logger LOG = LogManager.getLogger(InsertIntoTableCommand.class); + private LogicalPlan originalLogicalQuery; private LogicalPlan logicalQuery; private Optional labelName; /** * When source it's from job scheduler,it will be set. */ private long jobId; - private Optional insertCtx; + private final Optional insertCtx; private final Optional cte; /** @@ -89,7 +93,8 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync, public InsertIntoTableCommand(LogicalPlan logicalQuery, Optional labelName, Optional insertCtx, Optional cte) { super(PlanType.INSERT_INTO_TABLE_COMMAND); - this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null"); + this.originalLogicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery should not be null"); + this.logicalQuery = originalLogicalQuery; this.labelName = Objects.requireNonNull(labelName, "labelName should not be null"); this.insertCtx = insertCtx; this.cte = cte; @@ -145,19 +150,84 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); } - TableIf targetTableIf = InsertUtils.getTargetTable(logicalQuery, ctx); - // check auth - if (!Env.getCurrentEnv().getAccessManager() - .checkTblPriv(ConnectContext.get(), targetTableIf.getDatabase().getCatalog().getName(), - targetTableIf.getDatabase().getFullName(), targetTableIf.getName(), - PrivPredicate.LOAD)) { - ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", - ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), - targetTableIf.getDatabase().getFullName() + "." + targetTableIf.getName()); + List qualifiedTargetTableName = InsertUtils.getTargetTableQualified(logicalQuery, ctx); + AbstractInsertExecutor insertExecutor; + int retryTimes = 0; + while (++retryTimes < Math.max(ctx.getSessionVariable().dmlPlanRetryTimes, 3)) { + TableIf targetTableIf = RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); + // check auth + if (!Env.getCurrentEnv().getAccessManager() + .checkTblPriv(ConnectContext.get(), targetTableIf.getDatabase().getCatalog().getName(), + targetTableIf.getDatabase().getFullName(), targetTableIf.getName(), + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "LOAD", + ConnectContext.get().getQualifiedUser(), ConnectContext.get().getRemoteIP(), + targetTableIf.getDatabase().getFullName() + "." + targetTableIf.getName()); + } + BuildInsertExecutorResult buildResult; + try { + buildResult = initPlanOnce(ctx, executor, targetTableIf); + } catch (Throwable e) { + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + insertExecutor = buildResult.executor; + if (!needBeginTransaction) { + return insertExecutor; + } + + // lock after plan and check does table's schema changed to ensure we lock table order by id. + TableIf newestTargetTableIf = RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); + newestTargetTableIf.readLock(); + try { + if (targetTableIf.getId() != newestTargetTableIf.getId()) { + LOG.warn("insert plan failed {} times. query id is {}. table id changed from {} to {}", + retryTimes, DebugUtil.printId(ctx.queryId()), + targetTableIf.getId(), newestTargetTableIf.getId()); + continue; + } + if (!targetTableIf.getFullSchema().equals(newestTargetTableIf.getFullSchema())) { + LOG.warn("insert plan failed {} times. query id is {}. table schema changed from {} to {}", + retryTimes, DebugUtil.printId(ctx.queryId()), + targetTableIf.getFullSchema(), newestTargetTableIf.getFullSchema()); + continue; + } + if (ctx.getConnectType() == ConnectType.MYSQL && ctx.getMysqlChannel() != null) { + ctx.getMysqlChannel().reset(); + } + if (!insertExecutor.isEmptyInsert()) { + insertExecutor.beginTransaction(); + insertExecutor.finalizeSink( + buildResult.planner.getFragments().get(0), buildResult.dataSink, + buildResult.physicalSink + ); + } + newestTargetTableIf.readUnlock(); + } catch (Throwable e) { + newestTargetTableIf.readUnlock(); + // the abortTxn in onFail need to acquire table write lock + if (insertExecutor != null) { + insertExecutor.onFail(e); + } + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + executor.setProfileType(ProfileType.LOAD); + // We exposed @StmtExecutor#cancel as a unified entry point for statement interruption, + // so we need to set this here + insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); + executor.setCoord(insertExecutor.getCoordinator()); + // for prepare and execute, avoiding normalization for every execute command + this.originalLogicalQuery = this.logicalQuery; + return insertExecutor; } + LOG.warn("insert plan failed {} times. query id is {}.", retryTimes, DebugUtil.printId(ctx.queryId())); + throw new AnalysisException("Insert plan failed. Could not get target table lock."); + } - AbstractInsertExecutor insertExecutor = null; - // should lock target table until we begin transaction. + private BuildInsertExecutorResult initPlanOnce(ConnectContext ctx, + StmtExecutor stmtExecutor, TableIf targetTableIf) throws Exception { + AbstractInsertExecutor insertExecutor; targetTableIf.readLock(); try { // 1. process inline table (default values, empty values) @@ -165,98 +235,72 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor if (cte.isPresent()) { this.logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); } + } finally { + targetTableIf.readUnlock(); + } + LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery, ctx.getStatementContext()); + NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + stmtExecutor.setPlanner(planner); + stmtExecutor.checkBlockRules(); + Optional> plan = (planner.getPhysicalPlan() + .>collect(PhysicalSink.class::isInstance)).stream() + .findAny(); + Preconditions.checkArgument(plan.isPresent(), "insert into command must contain target table"); + PhysicalSink physicalSink = plan.get(); + DataSink sink = planner.getFragments().get(0).getSink(); + // Transaction insert should reuse the label in the transaction. + String label = this.labelName.orElse( + ctx.isTxnModel() ? null : String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); - LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery, ctx.getStatementContext()); - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); - executor.setPlanner(planner); - executor.checkBlockRules(); - if (ctx.getConnectType() == ConnectType.MYSQL && ctx.getMysqlChannel() != null) { - ctx.getMysqlChannel().reset(); + if (physicalSink instanceof PhysicalOlapTableSink) { + if (GroupCommitInserter.groupCommit(ctx, sink, physicalSink)) { + // return; + throw new AnalysisException("group commit is not supported in Nereids now"); } - Optional> plan = (planner.getPhysicalPlan() - .>collect(PhysicalSink.class::isInstance)).stream() - .findAny(); - Preconditions.checkArgument(plan.isPresent(), "insert into command must contain target table"); - PhysicalSink physicalSink = plan.get(); - DataSink sink = planner.getFragments().get(0).getSink(); - // Transaction insert should reuse the label in the transaction. - String label = this.labelName.orElse( - ctx.isTxnModel() ? null : String.format("label_%x_%x", ctx.queryId().hi, ctx.queryId().lo)); - - if (physicalSink instanceof PhysicalOlapTableSink) { - if (GroupCommitInserter.groupCommit(ctx, sink, physicalSink)) { - // return; - throw new AnalysisException("group commit is not supported in Nereids now"); - } - boolean emptyInsert = childIsEmptyRelation(physicalSink); - OlapTable olapTable = (OlapTable) targetTableIf; - // the insertCtx contains some variables to adjust SinkNode - insertExecutor = new OlapInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert); + boolean emptyInsert = childIsEmptyRelation(physicalSink); + OlapTable olapTable = (OlapTable) targetTableIf; + // the insertCtx contains some variables to adjust SinkNode + insertExecutor = new OlapInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert); - boolean isEnableMemtableOnSinkNode = - olapTable.getTableProperty().getUseSchemaLightChange() - ? insertExecutor.getCoordinator().getQueryOptions().isEnableMemtableOnSinkNode() - : false; - insertExecutor.getCoordinator().getQueryOptions() - .setEnableMemtableOnSinkNode(isEnableMemtableOnSinkNode); - } else if (physicalSink instanceof PhysicalHiveTableSink) { - boolean emptyInsert = childIsEmptyRelation(physicalSink); - HMSExternalTable hiveExternalTable = (HMSExternalTable) targetTableIf; - insertExecutor = new HiveInsertExecutor(ctx, hiveExternalTable, label, planner, - Optional.of(insertCtx.orElse((new HiveInsertCommandContext()))), emptyInsert); - // set hive query options - } else if (physicalSink instanceof PhysicalIcebergTableSink) { - boolean emptyInsert = childIsEmptyRelation(physicalSink); - IcebergExternalTable icebergExternalTable = (IcebergExternalTable) targetTableIf; - insertExecutor = new IcebergInsertExecutor(ctx, icebergExternalTable, label, planner, - Optional.of(insertCtx.orElse((new BaseExternalTableInsertCommandContext()))), emptyInsert); - } else if (physicalSink instanceof PhysicalJdbcTableSink) { - boolean emptyInsert = childIsEmptyRelation(physicalSink); - List cols = ((PhysicalJdbcTableSink) physicalSink).getCols(); - List slots = ((PhysicalJdbcTableSink) physicalSink).getOutput(); - if (physicalSink.children().size() == 1) { - if (physicalSink.child(0) instanceof PhysicalOneRowRelation - || physicalSink.child(0) instanceof PhysicalUnion) { - for (int i = 0; i < cols.size(); i++) { - if (!(cols.get(i).isAllowNull()) && slots.get(i).nullable()) { - throw new AnalysisException("Column `" + cols.get(i).getName() - + "` is not nullable, but the inserted value is nullable."); - } + boolean isEnableMemtableOnSinkNode = + olapTable.getTableProperty().getUseSchemaLightChange() + && insertExecutor.getCoordinator().getQueryOptions().isEnableMemtableOnSinkNode(); + insertExecutor.getCoordinator().getQueryOptions().setEnableMemtableOnSinkNode(isEnableMemtableOnSinkNode); + } else if (physicalSink instanceof PhysicalHiveTableSink) { + boolean emptyInsert = childIsEmptyRelation(physicalSink); + HMSExternalTable hiveExternalTable = (HMSExternalTable) targetTableIf; + insertExecutor = new HiveInsertExecutor(ctx, hiveExternalTable, label, planner, + Optional.of(insertCtx.orElse((new HiveInsertCommandContext()))), emptyInsert); + // set hive query options + } else if (physicalSink instanceof PhysicalIcebergTableSink) { + boolean emptyInsert = childIsEmptyRelation(physicalSink); + IcebergExternalTable icebergExternalTable = (IcebergExternalTable) targetTableIf; + insertExecutor = new IcebergInsertExecutor(ctx, icebergExternalTable, label, planner, + Optional.of(insertCtx.orElse((new BaseExternalTableInsertCommandContext()))), emptyInsert); + } else if (physicalSink instanceof PhysicalJdbcTableSink) { + boolean emptyInsert = childIsEmptyRelation(physicalSink); + List cols = ((PhysicalJdbcTableSink) physicalSink).getCols(); + List slots = physicalSink.getOutput(); + if (physicalSink.children().size() == 1) { + if (physicalSink.child(0) instanceof PhysicalOneRowRelation + || physicalSink.child(0) instanceof PhysicalUnion) { + for (int i = 0; i < cols.size(); i++) { + if (!(cols.get(i).isAllowNull()) && slots.get(i).nullable()) { + throw new AnalysisException("Column `" + cols.get(i).getName() + + "` is not nullable, but the inserted value is nullable."); } } } - JdbcExternalTable jdbcExternalTable = (JdbcExternalTable) targetTableIf; - insertExecutor = new JdbcInsertExecutor(ctx, jdbcExternalTable, label, planner, - Optional.of(insertCtx.orElse((new JdbcInsertCommandContext()))), emptyInsert); - } else { - // TODO: support other table types - throw new AnalysisException("insert into command only support [olap, hive, iceberg, jdbc] table"); - } - if (!needBeginTransaction) { - targetTableIf.readUnlock(); - return insertExecutor; } - if (!insertExecutor.isEmptyInsert()) { - insertExecutor.beginTransaction(); - insertExecutor.finalizeSink(planner.getFragments().get(0), sink, physicalSink); - } - targetTableIf.readUnlock(); - } catch (Throwable e) { - targetTableIf.readUnlock(); - // the abortTxn in onFail need to acquire table write lock - if (insertExecutor != null) { - insertExecutor.onFail(e); - } - throw e; + JdbcExternalTable jdbcExternalTable = (JdbcExternalTable) targetTableIf; + insertExecutor = new JdbcInsertExecutor(ctx, jdbcExternalTable, label, planner, + Optional.of(insertCtx.orElse((new JdbcInsertCommandContext()))), emptyInsert); + } else { + // TODO: support other table types + throw new AnalysisException("insert into command only support [olap, hive, iceberg, jdbc] table"); } - - executor.setProfileType(ProfileType.LOAD); - // We exposed @StmtExecutor#cancel as a unified entry point for statement interruption, - // so we need to set this here - insertExecutor.getCoordinator().setTxnId(insertExecutor.getTxnId()); - executor.setCoord(insertExecutor.getCoordinator()); - return insertExecutor; + return new BuildInsertExecutorResult(planner, insertExecutor, sink, physicalSink); } private void runInternal(ConnectContext ctx, StmtExecutor executor) throws Exception { @@ -289,4 +333,19 @@ private boolean childIsEmptyRelation(PhysicalSink sink) { } return false; } + + private static class BuildInsertExecutorResult { + private final NereidsPlanner planner; + private final AbstractInsertExecutor executor; + private final DataSink dataSink; + private final PhysicalSink physicalSink; + + public BuildInsertExecutorResult(NereidsPlanner planner, AbstractInsertExecutor executor, DataSink dataSink, + PhysicalSink physicalSink) { + this.planner = planner; + this.executor = executor; + this.dataSink = dataSink; + this.physicalSink = physicalSink; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java index d478192f1b430e..c77789bc0c85fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertUtils.java @@ -393,6 +393,14 @@ private static Expression castValue(Expression value, DataType targetType) { * get target table from names. */ public static TableIf getTargetTable(Plan plan, ConnectContext ctx) { + List tableQualifier = getTargetTableQualified(plan, ctx); + return RelationUtil.getTable(tableQualifier, ctx.getEnv()); + } + + /** + * get target table from names. + */ + public static List getTargetTableQualified(Plan plan, ConnectContext ctx) { UnboundLogicalSink unboundTableSink; if (plan instanceof UnboundTableSink) { unboundTableSink = (UnboundTableSink) plan; @@ -407,8 +415,7 @@ public static TableIf getTargetTable(Plan plan, ConnectContext ctx) { + " [UnboundTableSink, UnboundHiveTableSink, UnboundIcebergTableSink]," + " but it is " + plan.getType()); } - List tableQualifier = RelationUtil.getQualifierName(ctx, unboundTableSink.getNameParts()); - return RelationUtil.getDbAndTable(tableQualifier, ctx.getEnv()).second; + return RelationUtil.getQualifierName(ctx, unboundTableSink.getNameParts()); } private static NamedExpression generateDefaultExpression(Column column) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java deleted file mode 100644 index 27ff1e4b68c075..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java +++ /dev/null @@ -1,122 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.trees.plans.visitor; - -import org.apache.doris.catalog.MTMV; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.TableIf.TableType; -import org.apache.doris.common.AnalysisException; -import org.apache.doris.mtmv.MTMVCache; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; -import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; -import org.apache.doris.qe.ConnectContext; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.HashSet; -import java.util.Set; - -/** - * Collect the table in plan - * Note: will not get table if table is eliminated by EmptyRelation in rewrite. - * View expand is in RBO, if call this method with the plan after RBO, this will get base tables in view, or will not. - * Materialized view is extended or not can be controlled by the field expand - */ -public class TableCollector extends DefaultPlanVisitor { - - public static final TableCollector INSTANCE = new TableCollector(); - private static final Logger LOG = LogManager.getLogger(TableCollector.class); - - @Override - public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, TableCollectorContext context) { - TableIf table = catalogRelation.getTable(); - if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { - context.getCollectedTables().add(table); - } - if (table instanceof MTMV) { - expandMvAndCollect((MTMV) table, context); - } - return catalogRelation; - } - - @Override - public Plan visitPhysicalCatalogRelation(PhysicalCatalogRelation catalogRelation, TableCollectorContext context) { - TableIf table = catalogRelation.getTable(); - if (context.getTargetTableTypes().isEmpty() || context.getTargetTableTypes().contains(table.getType())) { - context.getCollectedTables().add(table); - } - if (table instanceof MTMV) { - expandMvAndCollect((MTMV) table, context); - } - return catalogRelation; - } - - private void expandMvAndCollect(MTMV mtmv, TableCollectorContext context) { - if (!context.isExpandMaterializedView()) { - return; - } - // Make sure use only one connection context when in query to avoid ConnectionContext.get() wrong - MTMVCache expandedMvCache; - try { - expandedMvCache = mtmv.getOrGenerateCache(context.getConnectContext()); - } catch (AnalysisException exception) { - LOG.warn(String.format("expandMvAndCollect getOrGenerateCache fail, mtmv name is %s", mtmv.getName()), - exception); - expandedMvCache = MTMVCache.from(mtmv, context.getConnectContext(), false); - } - expandedMvCache.getAnalyzedPlan().accept(this, context); - } - - /** - * The context for table collecting, it contains the target collect table types - * and the result of collect. - */ - public static final class TableCollectorContext { - private final Set collectedTables = new HashSet<>(); - private final Set targetTableTypes; - // if expand the mv or not - private final boolean expandMaterializedView; - private final ConnectContext connectContext; - - public TableCollectorContext(Set targetTableTypes, boolean expandMaterializedView, - ConnectContext connectContext) { - this.targetTableTypes = targetTableTypes; - this.expandMaterializedView = expandMaterializedView; - this.connectContext = connectContext; - } - - public Set getCollectedTables() { - return collectedTables; - } - - public Set getTargetTableTypes() { - return targetTableTypes; - } - - public boolean isExpandMaterializedView() { - return expandMaterializedView; - } - - public ConnectContext getConnectContext() { - return connectContext; - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 411f678490c179..9a22c3d96e1650 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -131,6 +131,7 @@ public class SessionVariable implements Serializable, Writable { public static final String PARALLEL_PIPELINE_TASK_NUM = "parallel_pipeline_task_num"; public static final String PROFILE_LEVEL = "profile_level"; public static final String MAX_INSTANCE_NUM = "max_instance_num"; + public static final String DML_PLAN_RETRY_TIMES = "DML_PLAN_RETRY_TIMES"; public static final String ENABLE_INSERT_STRICT = "enable_insert_strict"; public static final String INSERT_MAX_FILTER_RATIO = "insert_max_filter_ratio"; public static final String ENABLE_SPILLING = "enable_spilling"; @@ -966,6 +967,17 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = MAX_INSTANCE_NUM) public int maxInstanceNum = 64; + @VariableMgr.VarAttr(name = DML_PLAN_RETRY_TIMES, needForward = true, description = { + "写入规划的最大重试次数。为了避免死锁,写入规划时采用了分阶段加锁。当在两次加锁中间,表结构发生变更时,会尝试重新规划。" + + "此变量限制重新规划的最大尝试次数。", + "Maximum retry attempts for write planning. To avoid deadlocks, " + + "phased locking is adopted during write planning. " + + "When changes occur to the table structure between two locking phases, " + + "re-planning will be attempted. " + + "This variable limits the maximum number of retry attempts for re-planning." + }) + public int dmlPlanRetryTimes = 3; + @VariableMgr.VarAttr(name = ENABLE_INSERT_STRICT, needForward = true) public boolean enableInsertStrict = true; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index f314f3aa76ce14..e860395a167f7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -227,6 +227,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1266,6 +1267,7 @@ public void analyze(TQueryOptions tQueryOptions) throws UserException, Interrupt } // table id in tableList is in ascending order because that table map is a sorted map List tables = Lists.newArrayList(tableMap.values()); + tables.sort((Comparator.comparing(TableIf::getId))); int analyzeTimes = 2; for (int i = 1; i <= analyzeTimes; i++) { MetaLockUtils.readLockTables(tables); @@ -2285,6 +2287,7 @@ private void handleInsertStmt() throws Exception { response.getStatus(), i); if (i < maxRetry) { List tables = Lists.newArrayList(insertStmt.getTargetTable()); + tables.sort((Comparator.comparing(TableIf::getId))); MetaLockUtils.readLockTables(tables); try { insertStmt.reset(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index 752a3f236c9d16..8b7d799cd8fef0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -43,6 +43,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.proc.FrontendsProcNode; import org.apache.doris.common.proc.PartitionsProcDir; +import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.common.util.NetUtils; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; @@ -62,7 +63,10 @@ import org.apache.doris.job.common.JobType; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.task.AbstractTask; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVPartitionUtil; +import org.apache.doris.mtmv.MTMVStatus; +import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.plsql.metastore.PlsqlManager; import org.apache.doris.plsql.metastore.PlsqlProcedureKey; @@ -111,6 +115,7 @@ import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.Date; import java.util.List; import java.util.Map; @@ -844,22 +849,42 @@ private static TFetchSchemaTableDataResult mtmvMetadataResult(TMetadataTableRequ } MTMV mv = (MTMV) table; if (LOG.isDebugEnabled()) { - LOG.debug("mv: " + mv.toInfoString()); + LOG.debug("mv: {}", mv.toInfoString()); } + List needLocked = Lists.newArrayList(); + needLocked.add(mv); + boolean alwaysNotSync = false; + try { + for (BaseTableInfo baseTableInfo : mv.getRelation().getBaseTables()) { + TableIf baseTable = MTMVUtil.getTable(baseTableInfo); + needLocked.add(baseTable); + } + } catch (Exception e) { + alwaysNotSync = true; + } + needLocked.sort(Comparator.comparing(TableIf::getId)); + MetaLockUtils.readLockTables(needLocked); + boolean isSync; + try { + isSync = !alwaysNotSync && MTMVPartitionUtil.isMTMVSync(mv); + } finally { + MetaLockUtils.readUnlockTables(needLocked); + } + MTMVStatus mtmvStatus = mv.getStatus(); TRow trow = new TRow(); trow.addToColumnValue(new TCell().setLongVal(mv.getId())); trow.addToColumnValue(new TCell().setStringVal(mv.getName())); trow.addToColumnValue(new TCell().setStringVal(mv.getJobInfo().getJobName())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getState().name())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getSchemaChangeDetail())); - trow.addToColumnValue(new TCell().setStringVal(mv.getStatus().getRefreshState().name())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getState().name())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getSchemaChangeDetail())); + trow.addToColumnValue(new TCell().setStringVal(mtmvStatus.getRefreshState().name())); trow.addToColumnValue(new TCell().setStringVal(mv.getRefreshInfo().toString())); trow.addToColumnValue(new TCell().setStringVal(mv.getQuerySql())); trow.addToColumnValue(new TCell().setStringVal(mv.getMvProperties().toString())); trow.addToColumnValue(new TCell().setStringVal(mv.getMvPartitionInfo().toNameString())); - trow.addToColumnValue(new TCell().setBoolVal(MTMVPartitionUtil.isMTMVSync(mv))); + trow.addToColumnValue(new TCell().setBoolVal(isSync)); if (LOG.isDebugEnabled()) { - LOG.debug("mvend: " + mv.getName()); + LOG.debug("mv end: {}", mv.getName()); } dataBatch.add(trow); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java index 369a57017cba28..eaeaa3b2edda8b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java @@ -17,23 +17,13 @@ package org.apache.doris.nereids.rules.analysis; -import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.DatabaseIf; -import org.apache.doris.catalog.KeysType; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.PartitionInfo; -import org.apache.doris.catalog.RandomDistributionInfo; -import org.apache.doris.catalog.Type; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.pattern.GeneratedPlanPatterns; import org.apache.doris.nereids.rules.RulePromise; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanRewriter; import org.apache.doris.utframe.TestWithFeService; @@ -41,9 +31,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.List; -import java.util.Optional; - class BindRelationTest extends TestWithFeService implements GeneratedPlanPatterns { private static final String DB1 = "db1"; private static final String DB2 = "db2"; @@ -72,7 +59,7 @@ void bindInCurrentDb() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("t")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalOlapScan); + Assertions.assertInstanceOf(LogicalOlapScan.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "t"), ((LogicalOlapScan) plan).qualified()); @@ -84,63 +71,12 @@ void bindByDbQualifier() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("db1", "t")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalOlapScan); + Assertions.assertInstanceOf(LogicalOlapScan.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "t"), ((LogicalOlapScan) plan).qualified()); } - @Test - public void bindExternalRelation() { - connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB1); - String tableName = "external_table"; - - List externalTableColumns = ImmutableList.of( - new Column("id", Type.INT), - new Column("name", Type.VARCHAR) - ); - - Database externalDatabase = new Database(10000, DEFAULT_CLUSTER_PREFIX + DB1); - - OlapTable externalOlapTable = new OlapTable(1, tableName, externalTableColumns, KeysType.DUP_KEYS, - new PartitionInfo(), new RandomDistributionInfo(10)) { - @Override - public List getBaseSchema(boolean full) { - return externalTableColumns; - } - - @Override - public boolean hasDeleteSign() { - return false; - } - - @Override - public DatabaseIf getDatabase() { - return externalDatabase; - } - }; - - CustomTableResolver customTableResolver = qualifiedTable -> { - if (qualifiedTable.get(2).equals(tableName)) { - return externalOlapTable; - } else { - return null; - } - }; - - PlanChecker.from(connectContext) - .parse("select * from " + tableName + " as et join db1.t on et.id = t.a") - .customAnalyzer(Optional.of(customTableResolver)) // analyze internal relation - .matches( - logicalJoin( - logicalSubQueryAlias( - logicalOlapScan().when(r -> r.getTable() == externalOlapTable) - ), - logicalOlapScan().when(r -> r.getTable().getName().equals("t")) - ) - ); - } - @Test void bindRandomAggTable() { connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB1); @@ -148,7 +84,7 @@ void bindRandomAggTable() { Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("tagg")), connectContext, new BindRelation()); - Assertions.assertTrue(plan instanceof LogicalAggregate); + Assertions.assertInstanceOf(LogicalAggregate.class, plan); Assertions.assertEquals( ImmutableList.of("internal", DEFAULT_CLUSTER_PREFIX + DB1, "tagg"), plan.getOutput().get(0).getQualifier()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java index 9c56bb1186a5ed..d4809568bbec9f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanVisitorTest.java @@ -17,34 +17,19 @@ package org.apache.doris.nereids.trees.plans; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentDate; import org.apache.doris.nereids.trees.expressions.functions.scalar.CurrentTime; import org.apache.doris.nereids.trees.expressions.functions.scalar.Now; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Random; import org.apache.doris.nereids.trees.expressions.functions.scalar.UnixTimestamp; -import org.apache.doris.nereids.trees.expressions.functions.scalar.Uuid; -import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector; -import org.apache.doris.nereids.trees.plans.visitor.TableCollector.TableCollectorContext; import org.apache.doris.nereids.util.PlanChecker; -import org.apache.doris.qe.SessionVariable; import org.apache.doris.utframe.TestWithFeService; -import com.google.common.collect.Sets; -import mockit.Mock; -import mockit.MockUp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.BitSet; -import java.util.HashSet; import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; /** * Tests for plan visitors to make sure the result meets expectation. @@ -115,154 +100,6 @@ protected void runBeforeAll() throws Exception { + "inner join table3 t3 on t1.c1= t3.c2;"); } - @Test - public void test1() { - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT *, random() FROM table1 " - + "LEFT SEMI JOIN table2 ON table1.c1 = table2.c1 " - + "WHERE table1.c1 IN (SELECT c1 FROM table2) OR table1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(1, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Random); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - }); - } - - @Test - public void test2() { - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT view1.*, uuid() FROM view1 " - + "LEFT SEMI JOIN table2 ON view1.c1 = table2.c1 " - + "WHERE view1.c1 IN (SELECT c1 FROM table2) OR view1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(2, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Uuid); - Assertions.assertTrue(nondeterministicFunctionSet.get(1) instanceof Random); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - }); - } - - @Test - public void test3() throws Exception { - connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); - BitSet disableNereidsRules = connectContext.getSessionVariable().getDisableNereidsRules(); - new MockUp() { - @Mock - public BitSet getDisableNereidsRules() { - return disableNereidsRules; - } - }; - PlanChecker.from(connectContext) - .checkPlannerResult("SELECT mv1.*, uuid() FROM mv1 " - + "INNER JOIN view1 on mv1.c1 = view1.c2 " - + "LEFT SEMI JOIN table2 ON mv1.c1 = table2.c1 " - + "WHERE mv1.c1 IN (SELECT c1 FROM table2) OR mv1.c1 < 10", - nereidsPlanner -> { - PhysicalPlan physicalPlan = nereidsPlanner.getPhysicalPlan(); - // Check nondeterministic collect - List nondeterministicFunctionSet = - MaterializedViewUtils.extractNondeterministicFunction(physicalPlan); - Assertions.assertEquals(1, nondeterministicFunctionSet.size()); - Assertions.assertTrue(nondeterministicFunctionSet.get(0) instanceof Uuid); - // Check get tables - TableCollectorContext collectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.OLAP), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContext); - Set expectedTables = new HashSet<>(); - expectedTables.add("table1"); - expectedTables.add("table2"); - expectedTables.add("table3"); - Assertions.assertEquals( - collectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTables); - - TableCollectorContext collectorContextWithNoExpand = - new TableCollector.TableCollectorContext(Sets.newHashSet(TableType.OLAP), - false, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, collectorContextWithNoExpand); - Set expectedTablesWithNoExpand = new HashSet<>(); - expectedTablesWithNoExpand.add("table1"); - expectedTablesWithNoExpand.add("table2"); - Assertions.assertEquals( - collectorContextWithNoExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTablesWithNoExpand); - - TableCollectorContext mvCollectorContext = new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContext); - Set expectedMvs = new HashSet<>(); - expectedMvs.add("mv1"); - Assertions.assertEquals( - mvCollectorContext.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedMvs); - - TableCollectorContext mvCollectorContextWithNoExpand = - new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.MATERIALIZED_VIEW), false, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, mvCollectorContextWithNoExpand); - Set expectedMvsWithNoExpand = new HashSet<>(); - expectedMvsWithNoExpand.add("mv1"); - Assertions.assertEquals( - mvCollectorContextWithNoExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedMvsWithNoExpand); - - TableCollectorContext allTableTypeWithExpand = - new TableCollector.TableCollectorContext( - Sets.newHashSet(TableType.values()), true, connectContext); - physicalPlan.accept(TableCollector.INSTANCE, allTableTypeWithExpand); - // when collect in plan with expand, should collect table which is expended - Set expectedTablesWithExpand = new HashSet<>(); - expectedTablesWithExpand.add("mv1"); - expectedTablesWithExpand.add("table1"); - expectedTablesWithExpand.add("table2"); - expectedTablesWithExpand.add("table3"); - Assertions.assertEquals( - allTableTypeWithExpand.getCollectedTables().stream() - .map(TableIf::getName) - .collect(Collectors.toSet()), - expectedTablesWithExpand); - }); - dropMvByNereids("drop materialized view mv1"); - } - @Test public void testTimeFunction() { PlanChecker.from(connectContext) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java index b95027a1385009..e0cf7f66a02fe7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java @@ -49,7 +49,6 @@ import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.rules.exploration.mv.InitMaterializationContextHook; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -71,7 +70,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import java.util.function.Supplier; @@ -147,12 +145,6 @@ public PlanChecker analyze(String sql) { return this; } - public PlanChecker customAnalyzer(Optional customTableResolver) { - this.cascadesContext.newAnalyzer(customTableResolver).analyze(); - this.cascadesContext.toMemo(); - return this; - } - public PlanChecker customRewrite(CustomRewriter customRewriter) { Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, ImmutableList.of(Rewriter.custom(RuleType.TEST_REWRITE, () -> customRewriter))) @@ -565,7 +557,7 @@ public PlanChecker checkExplain(String sql, Consumer consumer) { new StatementContext(connectContext, new OriginStatement(sql, 0))); LogicalPlanAdapter adapter = LogicalPlanAdapter.of(parsed); adapter.setIsExplain(new ExplainOptions(ExplainLevel.ALL_PLAN, false)); - nereidsPlanner.planWithLock(adapter); + nereidsPlanner.plan(adapter); consumer.accept(nereidsPlanner); return this; } @@ -574,7 +566,7 @@ public PlanChecker checkPlannerResult(String sql, Consumer consu LogicalPlan parsed = new NereidsParser().parseSingle(sql); NereidsPlanner nereidsPlanner = new NereidsPlanner( new StatementContext(connectContext, new OriginStatement(sql, 0))); - nereidsPlanner.planWithLock(LogicalPlanAdapter.of(parsed)); + nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); consumer.accept(nereidsPlanner); return this; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java index 9283b286f417e4..c8ba1a7e83934c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ReadLockTest.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.util; import org.apache.doris.catalog.TableIf; -import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.datasets.ssb.SSBTestBase; @@ -30,9 +29,10 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; public class ReadLockTest extends SSBTestBase { @@ -47,11 +47,15 @@ public void testSimple() { parser.parseSingle(sql), PhysicalProperties.ANY ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); + Map, TableIf> f = statementContext.getTables(); Assertions.assertEquals(1, f.size()); - Assertions.assertEquals("supplier", f.stream().map(TableIf::getName).findFirst().get()); + Set tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } + Assertions.assertTrue(tableNames.contains("supplier")); } @Test @@ -69,10 +73,12 @@ public void testCTE() { parser.parseSingle(sql), PhysicalProperties.ANY ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); + Map, TableIf> f = statementContext.getTables(); Assertions.assertEquals(1, f.size()); - Assertions.assertEquals("supplier", f.stream().map(TableIf::getName).findFirst().get()); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + Assertions.assertEquals("supplier", table.getName()); + } } @Test @@ -84,10 +90,14 @@ public void testSubQuery() { parser.parseSingle(sql), PhysicalProperties.ANY ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); + Map, TableIf> f = statementContext.getTables(); Assertions.assertEquals(1, f.size()); - Assertions.assertEquals("supplier", f.stream().map(TableIf::getName).findFirst().get()); + Set tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } + Assertions.assertTrue(tableNames.contains("supplier")); } @Test @@ -99,10 +109,13 @@ public void testScalarSubQuery() { parser.parseSingle(sql), PhysicalProperties.ANY ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); + Map, TableIf> f = statementContext.getTables(); Assertions.assertEquals(2, f.size()); - Set tableNames = f.stream().map(TableIf::getName).collect(Collectors.toSet()); + Set tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } Assertions.assertTrue(tableNames.contains("supplier")); Assertions.assertTrue(tableNames.contains("lineorder")); } @@ -117,11 +130,21 @@ public void testInserInto() { (LogicalPlan) insertIntoTableCommand.getExplainPlan(connectContext), PhysicalProperties.ANY ); - CascadesContext cascadesContext = planner.getCascadesContext(); - List f = cascadesContext.getTables(); - Assertions.assertEquals(2, f.size()); - Set tableNames = f.stream().map(TableIf::getName).collect(Collectors.toSet()); - Assertions.assertTrue(tableNames.contains("supplier")); + Map, TableIf> f = statementContext.getTables(); + Assertions.assertEquals(1, f.size()); + Set tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } Assertions.assertTrue(tableNames.contains("lineorder")); + f = statementContext.getInsertTargetTables(); + Assertions.assertEquals(1, f.size()); + tableNames = new HashSet<>(); + for (Map.Entry, TableIf> entry : f.entrySet()) { + TableIf table = entry.getValue(); + tableNames.add(table.getName()); + } + Assertions.assertTrue(tableNames.contains("supplier")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java index 270d5a3152a563..0865a7b14f05af 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/OlapQueryCacheTest.java @@ -514,7 +514,7 @@ private StatementBase parseSqlByNereids(String sql) { ctx.setStatementContext(statementContext); NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); LogicalPlanAdapter adapter = new LogicalPlanAdapter(plan, statementContext); - nereidsPlanner.planWithLock(adapter); + nereidsPlanner.plan(adapter); statementContext.setParsedStatement(adapter); stmt = adapter; } catch (Throwable throwable) {