From 392268cca37fec51096a171575dd3a1f54c5a4d2 Mon Sep 17 00:00:00 2001 From: morrySnow Date: Thu, 19 Dec 2024 17:48:37 +0800 Subject: [PATCH] [opt](Nereids) lock table in ascending order of table IDs (#45045) Problem Summary: Doris's table locks are fair read-write locks. If two threads acquire read locks on tables in different orders and simultaneously a third thread attempts to acquire a write lock on one of these tables, a deadlock can form between the two threads trying to acquire read locks. This PR changes the lock acquisition order for queries to follow the order of table IDs, ensuring that the lock acquisition order for tables is consistent among different threads. Execute table locking operations in ascending order of table IDs --- .../java/org/apache/doris/catalog/Column.java | 5 +- .../java/org/apache/doris/catalog/MTMV.java | 6 +- .../org/apache/doris/catalog/OlapTable.java | 20 +- .../org/apache/doris/catalog/TableIf.java | 134 ++++----- .../doris/common/NereidsSqlCacheManager.java | 32 ++- .../lock/MonitoredReentrantReadWriteLock.java | 13 + .../doris/common/proc/PartitionsProcDir.java | 37 ++- .../doris/common/profile/SummaryProfile.java | 15 +- .../doris/job/extensions/mtmv/MTMVTask.java | 28 +- .../org/apache/doris/mtmv/BaseTableInfo.java | 7 + .../java/org/apache/doris/mtmv/MTMVCache.java | 30 ++- .../org/apache/doris/mtmv/MTMVJobManager.java | 11 +- .../apache/doris/mtmv/MTMVPartitionUtil.java | 5 +- .../org/apache/doris/mtmv/MTMVPlanUtil.java | 68 ++--- .../doris/mtmv/MTMVRelationManager.java | 21 +- .../apache/doris/mtmv/MTMVRewriteUtil.java | 4 +- .../apache/doris/nereids/CascadesContext.java | 254 +----------------- .../apache/doris/nereids/NereidsPlanner.java | 110 ++++---- .../apache/doris/nereids/SqlCacheContext.java | 4 + .../doris/nereids/StatementContext.java | 143 ++++++++-- .../UnboundBaseExternalTableSink.java | 6 - .../analyzer/UnboundOneRowRelation.java | 2 +- .../nereids/analyzer/UnboundRelation.java | 6 - .../nereids/analyzer/UnboundResultSink.java | 6 - .../nereids/analyzer/UnboundTVFRelation.java | 6 - .../nereids/analyzer/UnboundTableSink.java | 6 - .../doris/nereids/jobs/executor/Analyzer.java | 35 +-- .../nereids/jobs/executor/TableCollector.java | 71 +++++ .../doris/nereids/minidump/MinidumpUtils.java | 14 +- .../nereids/parser/LogicalPlanBuilder.java | 3 +- .../apache/doris/nereids/rules/RuleType.java | 9 +- .../nereids/rules/analysis/BindRelation.java | 57 ++-- .../rules/analysis/CollectRelation.java | 220 +++++++++++++++ .../mv/AsyncMaterializationContext.java | 5 - .../mv/InitMaterializationContextHook.java | 24 +- .../exploration/mv/MaterializedViewUtils.java | 8 +- .../plans/commands/AddConstraintCommand.java | 31 ++- .../trees/plans/commands/CommandUtils.java | 49 ---- .../plans/commands/DropConstraintCommand.java | 28 +- .../commands/ShowConstraintsCommand.java | 15 +- .../plans/commands/info/CreateMTMVInfo.java | 77 +++--- .../info/MTMVPartitionDefinition.java | 20 +- .../insert/InsertIntoTableCommand.java | 253 ++++++++++------- .../plans/commands/insert/InsertUtils.java | 11 +- .../trees/plans/visitor/TableCollector.java | 122 --------- .../org/apache/doris/qe/SessionVariable.java | 12 + .../org/apache/doris/qe/StmtExecutor.java | 3 + .../tablefunction/MetadataGenerator.java | 37 ++- .../rules/analysis/BindRelationTest.java | 70 +---- .../nereids/trees/plans/PlanVisitorTest.java | 163 ----------- .../doris/nereids/util/PlanChecker.java | 12 +- .../doris/nereids/util/ReadLockTest.java | 61 +++-- .../apache/doris/qe/OlapQueryCacheTest.java | 2 +- 53 files changed, 1149 insertions(+), 1242 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TableCollector.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CommandUtils.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/TableCollector.java 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) {