diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java index a0be94b683435c..ab0cec460e7117 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EnvFactory.java @@ -32,14 +32,21 @@ import org.apache.doris.load.loadv2.LoadJobScheduler; import org.apache.doris.load.loadv2.LoadManager; import org.apache.doris.load.routineload.RoutineLoadManager; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.stats.StatsErrorEstimator; +import org.apache.doris.nereids.trees.plans.distribute.DistributePlanner; +import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping; import org.apache.doris.planner.GroupCommitPlanner; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.Planner; import org.apache.doris.planner.ScanNode; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.Coordinator; +import org.apache.doris.qe.NereidsSqlCoordinator; import org.apache.doris.qe.OriginStatement; +import org.apache.doris.qe.SessionVariable; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.GlobalTransactionMgr; @@ -134,6 +141,9 @@ public BrokerLoadJob createBrokerLoadJob() { public Coordinator createCoordinator(ConnectContext context, Analyzer analyzer, Planner planner, StatsErrorEstimator statsErrorEstimator) { + if (planner instanceof NereidsPlanner && SessionVariable.canUseNereidsDistributePlanner()) { + return new NereidsSqlCoordinator(context, analyzer, (NereidsPlanner) planner, statsErrorEstimator); + } return new Coordinator(context, analyzer, planner, statsErrorEstimator); } @@ -141,8 +151,23 @@ public Coordinator createCoordinator(ConnectContext context, Analyzer analyzer, public Coordinator createCoordinator(Long jobId, TUniqueId queryId, DescriptorTable descTable, List fragments, List scanNodes, String timezone, boolean loadZeroTolerance, boolean enableProfile) { - return new Coordinator(jobId, queryId, descTable, fragments, scanNodes, timezone, loadZeroTolerance, - enableProfile); + if (SessionVariable.canUseNereidsDistributePlanner()) { + ConnectContext connectContext = new ConnectContext(); + connectContext.setQueryId(queryId); + StatementContext statementContext = new StatementContext( + connectContext, new OriginStatement("", 0) + ); + DistributePlanner distributePlanner = new DistributePlanner(statementContext, fragments); + FragmentIdMapping distributedPlans = distributePlanner.plan(); + + return new NereidsSqlCoordinator( + jobId, queryId, descTable, fragments, distributedPlans.valueList(), + scanNodes, timezone, loadZeroTolerance, enableProfile + ); + } + return new Coordinator( + jobId, queryId, descTable, fragments, scanNodes, timezone, loadZeroTolerance, enableProfile + ); } public GroupCommitPlanner createGroupCommitPlanner(Database db, OlapTable table, List targetColumnNames, 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 323566c5161109..ecc4c908809161 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 @@ -140,6 +140,7 @@ public class SummaryProfile { NEREIDS_REWRITE_TIME, NEREIDS_OPTIMIZE_TIME, NEREIDS_TRANSLATE_TIME, + NEREIDS_DISTRIBUTE_TIME, WORKLOAD_GROUP, ANALYSIS_TIME, PLAN_TIME, 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 4acae7164f83b0..26665275ff636b 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 @@ -228,7 +228,7 @@ public Plan planWithLock(LogicalPlan plan, PhysicalProperties requireProperties, } } - private Plan planWithoutLock( + protected Plan planWithoutLock( LogicalPlan plan, ExplainLevel explainLevel, boolean showPlanProcess, PhysicalProperties requireProperties) { // resolve column, table and function @@ -309,7 +309,7 @@ private Plan planWithoutLock( return physicalPlan; } - private LogicalPlan preprocess(LogicalPlan logicalPlan) { + protected LogicalPlan preprocess(LogicalPlan logicalPlan) { return new PlanPreprocessors(statementContext).process(logicalPlan); } @@ -320,7 +320,7 @@ private void initCascadesContext(LogicalPlan plan, PhysicalProperties requirePro } } - private void analyze(boolean showPlanProcess) { + protected void analyze(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("Start analyze plan"); } @@ -335,7 +335,7 @@ private void analyze(boolean showPlanProcess) { /** * Logical plan rewrite based on a series of heuristic rules. */ - private void rewrite(boolean showPlanProcess) { + protected void rewrite(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("Start rewrite plan"); } @@ -347,7 +347,7 @@ private void rewrite(boolean showPlanProcess) { } // DependsRules: EnsureProjectOnTopJoin.class - private void optimize() { + protected void optimize() { if (LOG.isDebugEnabled()) { LOG.debug("Start optimize plan"); } @@ -358,7 +358,7 @@ private void optimize() { } } - private void splitFragments(PhysicalPlan resultPlan) { + protected void splitFragments(PhysicalPlan resultPlan) { if (resultPlan instanceof PhysicalSqlCache) { return; } @@ -453,7 +453,7 @@ private void splitFragments(PhysicalPlan resultPlan) { } } - private void distribute(PhysicalPlan physicalPlan, ExplainLevel explainLevel) { + protected void distribute(PhysicalPlan physicalPlan, ExplainLevel explainLevel) { boolean canUseNereidsDistributePlanner = SessionVariable.canUseNereidsDistributePlanner(); if ((!canUseNereidsDistributePlanner && explainLevel.isPlanLevel)) { return; @@ -463,18 +463,21 @@ private void distribute(PhysicalPlan physicalPlan, ExplainLevel explainLevel) { } splitFragments(physicalPlan); + doDistribute(canUseNereidsDistributePlanner); + } + protected void doDistribute(boolean canUseNereidsDistributePlanner) { if (!canUseNereidsDistributePlanner) { return; } - distributedPlans = new DistributePlanner(fragments).plan(); + distributedPlans = new DistributePlanner(statementContext, fragments).plan(); if (statementContext.getConnectContext().getExecutor() != null) { statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsDistributeTime(); } } - private PhysicalPlan postProcess(PhysicalPlan physicalPlan) { + protected PhysicalPlan postProcess(PhysicalPlan physicalPlan) { return new PlanPostProcessors(cascadesContext).process(physicalPlan); } @@ -733,6 +736,10 @@ public CascadesContext getCascadesContext() { return cascadesContext; } + public ConnectContext getConnectContext() { + return cascadesContext.getConnectContext(); + } + public static PhysicalProperties buildInitRequireProperties() { return PhysicalProperties.GATHER; } 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 08e1e3fa815bf2..8edda551de139a 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 @@ -47,18 +47,19 @@ import org.apache.doris.qe.ShortCircuitQueryContext; import org.apache.doris.qe.cache.CacheAnalyzer; import org.apache.doris.statistics.Statistics; +import org.apache.doris.system.Backend; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.sparkproject.guava.base.Throwables; import java.io.Closeable; import java.util.ArrayList; @@ -169,6 +170,8 @@ public class StatementContext implements Closeable { private List plannerHooks = new ArrayList<>(); + private Backend groupCommitMergeBackend; + public StatementContext() { this(ConnectContext.get(), null, 0); } @@ -558,4 +561,13 @@ public TableId getTableId(TableIf tableIf) { this.tableIdMapping.put(tableIdentifier, tableId); return tableId; } + + public Backend getGroupCommitMergeBackend() { + return groupCommitMergeBackend; + } + + public void setGroupCommitMergeBackend( + Backend groupCommitMergeBackend) { + this.groupCommitMergeBackend = groupCommitMergeBackend; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java index aecb03a4158521..656fde3024e79f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java @@ -116,7 +116,6 @@ private void ensureChildrenRewritten() { // some rule return new plan tree, which the number of new plan node > 1, // we should transform this new plan nodes too. - // NOTICE: this relay on pull up cte anchor if (isTraverseChildren.test(plan)) { pushChildrenJobs(plan); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java index 0f87a745b5e43f..60969cdf6e701d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteJob.java @@ -60,8 +60,6 @@ protected final RewriteResult rewrite(Plan plan, List rules, RewriteJobCon } Plan newPlan = newPlans.get(0); if (!newPlan.deepEquals(plan)) { - // don't remove this comment, it can help us to trace some bug when developing. - NereidsTracer.logRewriteEvent(rule.toString(), pattern, plan, newPlan); String traceBefore = null; if (showPlanProcess) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java index 19d92e2f4f361a..806b2cf61ea454 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java @@ -58,7 +58,6 @@ public void execute() { RewriteJobContext newRewriteJobContext = rewriteJobContext.withChildrenVisited(true); pushJob(new PlanTreeRewriteTopDownJob(newRewriteJobContext, context, isTraverseChildren, rules)); - // NOTICE: this relay on pull up cte anchor if (isTraverseChildren.test(rewriteJobContext.plan)) { pushChildrenJobs(newRewriteJobContext); } 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 68718de0f86a5b..53e2c82f76a826 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 @@ -31,6 +31,7 @@ import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -53,15 +54,22 @@ import org.apache.doris.planner.DataSink; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ConnectContext.ConnectType; +import org.apache.doris.qe.Coordinator; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Backend; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +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.Objects; import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Supplier; /** * insert into select command implementation @@ -135,7 +143,7 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor * For external uses such as creating a job, only basic analysis is needed without starting a transaction, * in which case this can be set to false. */ - public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor, + public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExecutor, boolean needBeginTransaction) throws Exception { TableIf targetTableIf = InsertUtils.getTargetTable(logicalQuery, ctx); // check auth @@ -159,10 +167,50 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor } OlapGroupCommitInsertExecutor.analyzeGroupCommit(ctx, targetTableIf, this.logicalQuery, this.insertCtx); LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalQuery, ctx.getStatementContext()); - NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext()); - planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); - executor.setPlanner(planner); - executor.checkBlockRules(); + + BuildInsertExecutorResult buildResult = planInsertExecutor( + ctx, stmtExecutor, logicalPlanAdapter, targetTableIf + ); + + insertExecutor = buildResult.executor; + + if (!needBeginTransaction) { + targetTableIf.readUnlock(); + return insertExecutor; + } + if (!insertExecutor.isEmptyInsert()) { + insertExecutor.beginTransaction(); + insertExecutor.finalizeSink( + buildResult.planner.getFragments().get(0), buildResult.dataSink, buildResult.physicalSink + ); + } + targetTableIf.readUnlock(); + } catch (Throwable e) { + targetTableIf.readUnlock(); + // the abortTxn in onFail need to acquire table write lock + if (insertExecutor != null) { + insertExecutor.onFail(e); + } + Throwables.propagateIfInstanceOf(e, RuntimeException.class); + throw new IllegalStateException(e.getMessage(), e); + } + + stmtExecutor.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()); + stmtExecutor.setCoord(insertExecutor.getCoordinator()); + return insertExecutor; + } + + // we should select the factory type first, but we can not initial InsertExecutor at this time, + // because Nereids's DistributePlan are not gernerated, so we return factory and after the + // DistributePlan have been generated, we can create InsertExecutor + private ExecutorFactory selectInsertExecutorFactory( + NereidsPlanner planner, ConnectContext ctx, StmtExecutor stmtExecutor, TableIf targetTableIf) { + try { + stmtExecutor.setPlanner(planner); + stmtExecutor.checkBlockRules(); if (ctx.getConnectType() == ConnectType.MYSQL && ctx.getMysqlChannel() != null) { ctx.getMysqlChannel().reset(); } @@ -170,8 +218,8 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor .>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(); + PhysicalSink physicalSink = plan.get(); + DataSink dataSink = 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)); @@ -179,37 +227,72 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor if (physicalSink instanceof PhysicalOlapTableSink) { boolean emptyInsert = childIsEmptyRelation(physicalSink); OlapTable olapTable = (OlapTable) targetTableIf; + + ExecutorFactory executorFactory; // the insertCtx contains some variables to adjust SinkNode if (ctx.isTxnModel()) { - insertExecutor = new OlapTxnInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert); + executorFactory = ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> new OlapTxnInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert) + ); } else if (ctx.isGroupCommit()) { - insertExecutor = new OlapGroupCommitInsertExecutor(ctx, olapTable, label, planner, insertCtx, - emptyInsert); + Backend groupCommitBackend = Env.getCurrentEnv() + .getGroupCommitManager() + .selectBackendForGroupCommit(targetTableIf.getId(), ctx); + // set groupCommitBackend for Nereids's DistributePlanner + planner.getCascadesContext().getStatementContext().setGroupCommitMergeBackend(groupCommitBackend); + executorFactory = ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> new OlapGroupCommitInsertExecutor( + ctx, olapTable, label, planner, insertCtx, emptyInsert, groupCommitBackend + ) + ); } else { - insertExecutor = new OlapInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert); + executorFactory = ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> new OlapInsertExecutor(ctx, olapTable, label, planner, insertCtx, emptyInsert) + ); } - boolean isEnableMemtableOnSinkNode = - olapTable.getTableProperty().getUseSchemaLightChange() - ? insertExecutor.getCoordinator().getQueryOptions().isEnableMemtableOnSinkNode() - : false; - insertExecutor.getCoordinator().getQueryOptions() - .setEnableMemtableOnSinkNode(isEnableMemtableOnSinkNode); + return executorFactory.onCreate(executor -> { + Coordinator coordinator = executor.getCoordinator(); + boolean isEnableMemtableOnSinkNode = olapTable.getTableProperty().getUseSchemaLightChange() + && coordinator.getQueryOptions().isEnableMemtableOnSinkNode(); + coordinator.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); + return ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> 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); + return ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> 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(); + List slots = physicalSink.getOutput(); if (physicalSink.children().size() == 1) { if (physicalSink.child(0) instanceof PhysicalOneRowRelation || physicalSink.child(0) instanceof PhysicalUnion) { @@ -222,36 +305,58 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor executor } } JdbcExternalTable jdbcExternalTable = (JdbcExternalTable) targetTableIf; - insertExecutor = new JdbcInsertExecutor(ctx, jdbcExternalTable, label, planner, - Optional.of(insertCtx.orElse((new JdbcInsertCommandContext()))), emptyInsert); + return ExecutorFactory.from( + planner, + dataSink, + physicalSink, + () -> 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; + } catch (Throwable t) { + Throwables.propagateIfInstanceOf(t, RuntimeException.class); + throw new IllegalStateException(t.getMessage(), t); } + } - 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; + private BuildInsertExecutorResult planInsertExecutor( + ConnectContext ctx, StmtExecutor stmtExecutor, + LogicalPlanAdapter logicalPlanAdapter, TableIf targetTableIf) throws Throwable { + // the key logical when use new coordinator: + // 1. use NereidsPlanner to generate PhysicalPlan + // 2. use PhysicalPlan to select InsertExecutorFactory, some InsertExecutors want to control + // which backend should be used, for example, OlapGroupCommitInsertExecutor need select + // a backend to do group commit. + // Note: we can not initialize InsertExecutor at this time, because the DistributePlans + // have not been generated, so the NereidsSqlCoordinator can not initial too, + // 3. NereidsPlanner use PhysicalPlan and the provided backend to generate DistributePlan + // 4. ExecutorFactory use the DistributePlan to generate the NereidsSqlCoordinator and InsertExecutor + + StatementContext statementContext = ctx.getStatementContext(); + + AtomicReference executorFactoryRef = new AtomicReference<>(); + NereidsPlanner planner = new NereidsPlanner(statementContext) { + @Override + protected void doDistribute(boolean canUseNereidsDistributePlanner) { + // when enter this method, the step 1 already executed + + // step 2 + executorFactoryRef.set( + selectInsertExecutorFactory(this, ctx, stmtExecutor, targetTableIf) + ); + // step 3 + super.doDistribute(canUseNereidsDistributePlanner); + } + }; + + // step 1, 2, 3 + planner.plan(logicalPlanAdapter, ctx.getSessionVariable().toThrift()); + + // step 4 + return executorFactoryRef.get().build(); } private void runInternal(ConnectContext ctx, StmtExecutor executor) throws Exception { @@ -289,4 +394,59 @@ private boolean childIsEmptyRelation(PhysicalSink sink) { public StmtType stmtType() { return StmtType.INSERT; } + + /** + * this factory is used to delay create the AbstractInsertExecutor until the DistributePlan is generated + * by NereidsPlanner + */ + private static class ExecutorFactory { + public final NereidsPlanner planner; + public final DataSink dataSink; + public final PhysicalSink physicalSink; + public final Supplier executorSupplier; + private List> createCallback; + + private ExecutorFactory(NereidsPlanner planner, DataSink dataSink, PhysicalSink physicalSink, + Supplier executorSupplier) { + this.planner = planner; + this.dataSink = dataSink; + this.physicalSink = physicalSink; + this.executorSupplier = executorSupplier; + this.createCallback = Lists.newArrayList(); + } + + public static ExecutorFactory from( + NereidsPlanner planner, DataSink dataSink, PhysicalSink physicalSink, + Supplier executorSupplier) { + return new ExecutorFactory(planner, dataSink, physicalSink, executorSupplier); + } + + public ExecutorFactory onCreate(Consumer onCreate) { + this.createCallback.add(onCreate); + return this; + } + + public BuildInsertExecutorResult build() { + AbstractInsertExecutor executor = executorSupplier.get(); + for (Consumer callback : createCallback) { + callback.accept(executor); + } + return new BuildInsertExecutorResult(planner, executor, dataSink, physicalSink); + } + } + + 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/OlapGroupCommitInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapGroupCommitInsertExecutor.java index 239328ce93d813..e7b1f4d581892c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapGroupCommitInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapGroupCommitInsertExecutor.java @@ -22,10 +22,8 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.catalog.TableIf; -import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.FeConstants; -import org.apache.doris.common.LoadException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.mtmv.MTMVUtil; @@ -39,6 +37,7 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QueryState.MysqlStateType; import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.system.Backend; import org.apache.doris.transaction.TransactionStatus; import com.google.common.base.Strings; @@ -57,10 +56,13 @@ public class OlapGroupCommitInsertExecutor extends OlapInsertExecutor { private static final Logger LOG = LogManager.getLogger(OlapGroupCommitInsertExecutor.class); + private Backend groupCommitBackend; + public OlapGroupCommitInsertExecutor(ConnectContext ctx, Table table, String labelName, NereidsPlanner planner, Optional insertCtx, - boolean emptyInsert) { + boolean emptyInsert, Backend backend) { super(ctx, table, labelName, planner, insertCtx, emptyInsert); + this.groupCommitBackend = backend; } protected static void analyzeGroupCommit(ConnectContext ctx, TableIf table, LogicalPlan logicalQuery, @@ -118,12 +120,8 @@ protected void beforeExec() { LOG.info(msg); throw new AnalysisException(msg); } - try { - this.coordinator.setGroupCommitBe(Env.getCurrentEnv().getGroupCommitManager() - .selectBackendForGroupCommit(table.getId(), ctx)); - } catch (LoadException | DdlException e) { - throw new RuntimeException(e); - } + // this is used for old coordinator + this.coordinator.setGroupCommitBe(groupCommitBackend); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java index 658b154b017167..af2201d9dbbecf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/OlapInsertExecutor.java @@ -75,12 +75,15 @@ public class OlapInsertExecutor extends AbstractInsertExecutor { private static final Logger LOG = LogManager.getLogger(OlapInsertExecutor.class); protected TransactionStatus txnStatus = TransactionStatus.ABORTED; + protected OlapTable olapTable; + /** * constructor */ public OlapInsertExecutor(ConnectContext ctx, Table table, String labelName, NereidsPlanner planner, Optional insertCtx, boolean emptyInsert) { super(ctx, table, labelName, planner, insertCtx, emptyInsert); + this.olapTable = (OlapTable) table; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java index ceef281c0fc534..12ab8b42eaab61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java @@ -17,34 +17,74 @@ package org.apache.doris.nereids.trees.plans.distribute; +import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DummyWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJobBuilder; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.LocalShuffleAssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.StaticAssignedJob; import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJobBuilder; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedScanBucketOlapTableJob; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.DataStreamSink; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.MultiCastDataSink; +import org.apache.doris.planner.MultiCastPlanFragment; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.PlanFragmentId; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.thrift.TPartitionType; +import org.apache.doris.thrift.TUniqueId; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.LinkedHashMultimap; import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.SetMultimap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.Optional; +import java.util.function.Consumer; /** DistributePlanner */ public class DistributePlanner { - private final List fragments; + private static final Logger LOG = LogManager.getLogger(DistributePlanner.class); + private final StatementContext statementContext; private final FragmentIdMapping idToFragments; - public DistributePlanner(List fragments) { - this.fragments = Objects.requireNonNull(fragments, "fragments can not be null"); + public DistributePlanner(StatementContext statementContext, List fragments) { + this.statementContext = Objects.requireNonNull(statementContext, "statementContext can not be null"); this.idToFragments = FragmentIdMapping.buildFragmentMapping(fragments); } + /** plan */ public FragmentIdMapping plan() { - FragmentIdMapping fragmentJobs = UnassignedJobBuilder.buildJobs(idToFragments); - ListMultimap instanceJobs = AssignedJobBuilder.buildJobs(fragmentJobs); - return buildDistributePlans(fragmentJobs, instanceJobs); + try { + FragmentIdMapping fragmentJobs + = UnassignedJobBuilder.buildJobs(statementContext, idToFragments); + ListMultimap instanceJobs = AssignedJobBuilder.buildJobs(fragmentJobs); + FragmentIdMapping distributedPlans = buildDistributePlans(fragmentJobs, instanceJobs); + FragmentIdMapping linkedPlans = linkPlans(distributedPlans); + updateProfileIfPresent(SummaryProfile::setAssignFragmentTime); + return linkedPlans; + } catch (Throwable t) { + LOG.error("Failed to build distribute plans", t); + throw t; + } } private FragmentIdMapping buildDistributePlans( @@ -58,9 +98,151 @@ private FragmentIdMapping buildDistributePlans( UnassignedJob fragmentJob = idToUnassignedJobs.get(fragmentId); List instanceJobs = idToAssignedJobs.get(fragmentId); - List childrenPlans = idToDistributedPlans.getByChildrenFragments(fragment); - idToDistributedPlans.put(fragmentId, new PipelineDistributedPlan(fragmentJob, instanceJobs, childrenPlans)); + SetMultimap exchangeNodeToChildren = LinkedHashMultimap.create(); + for (PlanFragment childFragment : fragment.getChildren()) { + if (childFragment instanceof MultiCastPlanFragment) { + for (ExchangeNode exchangeNode : ((MultiCastPlanFragment) childFragment).getDestNodeList()) { + if (exchangeNode.getFragment() == fragment) { + exchangeNodeToChildren.put( + exchangeNode, idToDistributedPlans.get(childFragment.getFragmentId()) + ); + } + } + } else { + exchangeNodeToChildren.put( + childFragment.getDestNode(), + idToDistributedPlans.get(childFragment.getFragmentId()) + ); + } + } + + idToDistributedPlans.put(fragmentId, + new PipelineDistributedPlan(fragmentJob, instanceJobs, exchangeNodeToChildren) + ); } return (FragmentIdMapping) idToDistributedPlans; } + + private FragmentIdMapping linkPlans(FragmentIdMapping plans) { + boolean enableShareHashTableForBroadcastJoin = statementContext.getConnectContext() + .getSessionVariable() + .enableShareHashTableForBroadcastJoin; + for (DistributedPlan receiverPlan : plans.values()) { + for (Entry link : receiverPlan.getInputs().entries()) { + linkPipelinePlan( + (PipelineDistributedPlan) receiverPlan, + (PipelineDistributedPlan) link.getValue(), + link.getKey(), + enableShareHashTableForBroadcastJoin + ); + } + } + return plans; + } + + // set shuffle destinations + private void linkPipelinePlan( + PipelineDistributedPlan receiverPlan, + PipelineDistributedPlan senderPlan, + ExchangeNode linkNode, + boolean enableShareHashTableForBroadcastJoin) { + + List receiverInstances = filterInstancesWhichCanReceiveDataFromRemote( + receiverPlan, enableShareHashTableForBroadcastJoin, linkNode); + if (linkNode.getPartitionType() == TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED) { + receiverInstances = getDestinationsByBuckets(receiverPlan, receiverInstances); + } + + DataSink sink = senderPlan.getFragmentJob().getFragment().getSink(); + if (sink instanceof MultiCastDataSink) { + MultiCastDataSink multiCastDataSink = (MultiCastDataSink) sink; + receiverPlan.getFragmentJob().getFragment().setOutputPartition(multiCastDataSink.getOutputPartition()); + for (DataStreamSink realSink : multiCastDataSink.getDataStreamSinks()) { + if (realSink.getExchNodeId() == linkNode.getId()) { + senderPlan.addDestinations(realSink, receiverInstances); + break; + } + } + } else { + senderPlan.addDestinations(sink, receiverInstances); + } + } + + private List getDestinationsByBuckets( + PipelineDistributedPlan joinSide, + List receiverInstances) { + UnassignedScanBucketOlapTableJob bucketJob = (UnassignedScanBucketOlapTableJob) joinSide.getFragmentJob(); + int bucketNum = bucketJob.getOlapScanNodes().get(0).getBucketNum(); + return sortDestinationInstancesByBuckets(joinSide, receiverInstances, bucketNum); + } + + private List filterInstancesWhichCanReceiveDataFromRemote( + PipelineDistributedPlan receiverPlan, + boolean enableShareHashTableForBroadcastJoin, + ExchangeNode linkNode) { + boolean useLocalShuffle = receiverPlan.getInstanceJobs().stream() + .anyMatch(LocalShuffleAssignedJob.class::isInstance); + if (useLocalShuffle) { + return getFirstInstancePerShareScan(receiverPlan); + } else if (enableShareHashTableForBroadcastJoin && linkNode.isRightChildOfBroadcastHashJoin()) { + return getFirstInstancePerWorker(receiverPlan.getInstanceJobs()); + } else { + return receiverPlan.getInstanceJobs(); + } + } + + private List sortDestinationInstancesByBuckets( + PipelineDistributedPlan plan, List unsorted, int bucketNum) { + AssignedJob[] instances = new AssignedJob[bucketNum]; + for (AssignedJob instanceJob : unsorted) { + BucketScanSource bucketScanSource = (BucketScanSource) instanceJob.getScanSource(); + for (Integer bucketIndex : bucketScanSource.bucketIndexToScanNodeToTablets.keySet()) { + if (instances[bucketIndex] != null) { + throw new IllegalStateException( + "Multi instances scan same buckets: " + instances[bucketIndex] + " and " + instanceJob + ); + } + instances[bucketIndex] = instanceJob; + } + } + + for (int i = 0; i < instances.length; i++) { + if (instances[i] == null) { + instances[i] = new StaticAssignedJob( + i, + new TUniqueId(-1, -1), + plan.getFragmentJob(), + DummyWorker.INSTANCE, + new DefaultScanSource(ImmutableMap.of()) + ); + } + } + return Arrays.asList(instances); + } + + private List getFirstInstancePerShareScan(PipelineDistributedPlan plan) { + List canReceiveDataFromRemote = Lists.newArrayListWithCapacity(plan.getInstanceJobs().size()); + for (AssignedJob instanceJob : plan.getInstanceJobs()) { + LocalShuffleAssignedJob localShuffleJob = (LocalShuffleAssignedJob) instanceJob; + if (!localShuffleJob.receiveDataFromLocal) { + canReceiveDataFromRemote.add(localShuffleJob); + } + } + return canReceiveDataFromRemote; + } + + private List getFirstInstancePerWorker(List instances) { + Map firstInstancePerWorker = Maps.newLinkedHashMap(); + for (AssignedJob instance : instances) { + firstInstancePerWorker.putIfAbsent(instance.getAssignedWorker(), instance); + } + return Utils.fastToImmutableList(firstInstancePerWorker.values()); + } + + private void updateProfileIfPresent(Consumer profileAction) { + Optional.ofNullable(ConnectContext.get()) + .map(ConnectContext::getExecutor) + .map(StmtExecutor::getSummaryProfile) + .ifPresent(profileAction); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java index 8f176e3caccc7c..5b934bf7bfe534 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java @@ -19,7 +19,9 @@ import org.apache.doris.nereids.trees.AbstractTreeNode; import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; -import org.apache.doris.nereids.util.Utils; +import org.apache.doris.planner.ExchangeNode; + +import com.google.common.collect.SetMultimap; import java.util.List; import java.util.Objects; @@ -28,11 +30,19 @@ @lombok.Getter public abstract class DistributedPlan extends AbstractTreeNode { protected final UnassignedJob fragmentJob; - protected final List inputs; + protected final SetMultimap inputs; - public DistributedPlan(UnassignedJob fragmentJob, List inputs) { + public DistributedPlan(UnassignedJob fragmentJob, SetMultimap inputs) { this.fragmentJob = Objects.requireNonNull(fragmentJob, "fragmentJob can not be null"); - this.inputs = Utils.fastToImmutableList(Objects.requireNonNull(inputs, "inputs can not be null")); + this.inputs = Objects.requireNonNull(inputs, "inputs can not be null"); + } + + public UnassignedJob getFragmentJob() { + return fragmentJob; + } + + public SetMultimap getInputs() { + return inputs; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java index 95bf36051d2033..12845fbea480aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute; +import org.apache.doris.nereids.util.Utils; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.PlanFragmentId; @@ -68,4 +69,13 @@ public static FragmentIdMapping buildFragmentMapping(List List valueList() { + return (List) Utils.fastToImmutableList(values()); + } + + public T last() { + List valueList = valueList(); + return valueList.get(valueList.size() - 1); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java index 13f903d8a2cf05..37a00bd0052607 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java @@ -17,32 +17,69 @@ package org.apache.doris.nereids.trees.plans.distribute; +import org.apache.doris.common.util.DebugUtil; import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.LocalShuffleAssignedJob; import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.ExchangeNode; import org.apache.doris.thrift.TExplainLevel; +import com.google.common.collect.Maps; +import com.google.common.collect.SetMultimap; + import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; /** PipelineDistributedPlan */ public class PipelineDistributedPlan extends DistributedPlan { protected final List instanceJobs; + // current, we only support all instances of the same fragment reuse the same destination + private Map> destinations; + /** constructor */ public PipelineDistributedPlan( UnassignedJob fragmentJob, List instanceJobs, - List inputs) { + SetMultimap inputs) { super(fragmentJob, inputs); + + long localShuffleInstanceNum = instanceJobs.stream() + .filter(LocalShuffleAssignedJob.class::isInstance) + .count(); + if (localShuffleInstanceNum != 0 && localShuffleInstanceNum != instanceJobs.size()) { + throw new IllegalStateException("LocalShuffleAssignedJob num is " + localShuffleInstanceNum + + ", should be 0 or " + instanceJobs.size() + ", fragmentJob: " + fragmentJob + + ", instances: " + instanceJobs); + } + this.instanceJobs = Utils.fastToImmutableList( Objects.requireNonNull(instanceJobs, "instanceJobs can not be null") ); + this.destinations = Maps.newLinkedHashMap(); } public List getInstanceJobs() { return instanceJobs; } + public Map> getDestinations() { + return destinations; + } + + public void addDestinations(DataSink sink, List destinations) { + this.destinations.put(sink, destinations); + } + + @Override + public int hashCode() { + return fragmentJob.getFragment().getFragmentId().asInt(); + } + @Override public String toString(int displayFragmentId) { StringBuilder instancesStr = new StringBuilder(); @@ -58,10 +95,25 @@ public String toString(int displayFragmentId) { fragmentJob.getFragment().getExplainString(TExplainLevel.VERBOSE).trim(), " " ); + String destinationStr = destinations.entrySet() + .stream() + .map(kv -> { + AtomicInteger bucketNum = new AtomicInteger(0); + String str = kv.getValue() + .stream() + .map(destination -> " " + + "#" + bucketNum.getAndIncrement() + ": " + + DebugUtil.printId(destination.instanceId())) + .collect(Collectors.joining(",\n")); + return " Exchange " + kv.getKey().getExchNodeId().asInt() + + ": [" + (str.isEmpty() ? "" : "\n" + str + "\n ") + "]"; + }) + .collect(Collectors.joining(",\n")); return "PipelineDistributedPlan(\n" + " id: " + displayFragmentId + ",\n" + " parallel: " + instanceJobs.size() + ",\n" + " fragmentJob: " + fragmentJob + ",\n" + + " destinations: [" + (destinationStr.isEmpty() ? "" : "\n" + destinationStr + "\n ") + "],\n" + " fragment: {\n" + " " + explainString + "\n" + " },\n" diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java index 7acbe653e983d5..ed01d93993b092 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java @@ -30,7 +30,7 @@ /** BackendWorkerManager */ public class BackendDistributedPlanWorkerManager implements DistributedPlanWorkerManager { - private final Supplier> backends = Suppliers.memoize(() -> { + private final Supplier> allClusterBackends = Suppliers.memoize(() -> { try { return Env.getCurrentSystemInfo().getAllBackendsByAllCluster(); } catch (Exception t) { @@ -38,9 +38,21 @@ public class BackendDistributedPlanWorkerManager implements DistributedPlanWorke } }); + private final Supplier> currentClusterBackends = Suppliers.memoize(() -> { + try { + return Env.getCurrentSystemInfo().getBackendsByCurrentCluster(); + } catch (Exception t) { + throw new NereidsException("Can not get backends: " + t, t); + } + }); + + public boolean isCurrentClusterBackend(long backendId) { + return currentClusterBackends.get().containsKey(backendId); + } + @Override public DistributedPlanWorker getWorker(long backendId) { - ImmutableMap backends = this.backends.get(); + ImmutableMap backends = this.allClusterBackends.get(); Backend backend = backends.get(backendId); if (backend == null) { throw new IllegalStateException("Backend " + backendId + " is not exist"); @@ -52,7 +64,7 @@ public DistributedPlanWorker getWorker(long backendId) { public DistributedPlanWorker randomAvailableWorker() { try { Reference selectedBackendId = new Reference<>(); - ImmutableMap backends = this.backends.get(); + ImmutableMap backends = this.currentClusterBackends.get(); SimpleScheduler.getHost(backends, selectedBackendId); Backend selctedBackend = backends.get(selectedBackendId.getRef()); return new BackendWorker(selctedBackend); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java index 702a00dd358d29..63c73b50edcd07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java @@ -26,7 +26,11 @@ public class BackendWorker implements DistributedPlanWorker { private final Backend backend; public BackendWorker(Backend backend) { - this.backend = backend; + this.backend = Objects.requireNonNull(backend, "backend can not be null"); + } + + public Backend getBackend() { + return backend; } @Override @@ -39,6 +43,16 @@ public String address() { return backend.getAddress(); } + @Override + public String brpcAddress() { + return backend.getHost() + brpcPort(); + } + + @Override + public int brpcPort() { + return backend.getBrpcPort(); + } + @Override public String host() { return backend.getHost(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java index c86675a6dab27c..79f8b482d88c2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java @@ -30,6 +30,10 @@ public interface DistributedPlanWorker extends Comparable int port(); + String brpcAddress(); + + int brpcPort(); + // whether is this worker alive? boolean available(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DummyWorker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DummyWorker.java new file mode 100644 index 00000000000000..9a7d2f42476233 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DummyWorker.java @@ -0,0 +1,60 @@ +// 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.distribute.worker; + +/** DummyWorker */ +public class DummyWorker implements DistributedPlanWorker { + public static final DummyWorker INSTANCE = new DummyWorker(); + + private DummyWorker() {} + + @Override + public long id() { + return 0; + } + + @Override + public String address() { + return "0.0.0.0:0"; + } + + @Override + public String host() { + return "0.0.0.0"; + } + + @Override + public int port() { + return 0; + } + + @Override + public String brpcAddress() { + return "0.0.0.0:0"; + } + + @Override + public int brpcPort() { + return 0; + } + + @Override + public boolean available() { + return false; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java index 4cec3af18da182..89931daefe8253 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java @@ -29,6 +29,7 @@ import org.apache.doris.planner.OlapScanNode; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TExternalScanRange; import org.apache.doris.thrift.TFileRangeDesc; @@ -40,10 +41,13 @@ import org.apache.doris.thrift.TScanRangeParams; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -75,16 +79,24 @@ public DistributedPlanWorker selectMinWorkloadWorker(List } @Override - public Map selectReplicaAndWorkerWithoutBucket(ScanNode scanNode) { + public Map selectReplicaAndWorkerWithoutBucket( + ScanNode scanNode, ConnectContext context) { + Map workerScanRanges = Maps.newLinkedHashMap(); // allScanRangesLocations is all scan ranges in all partition which need to scan List allScanRangesLocations = scanNode.getScanRangeLocations(0); + + boolean orderedScanRangeLocations = shouldSortTablets(ImmutableList.of(scanNode), context); + if (orderedScanRangeLocations) { + allScanRangesLocations = sortScanRanges(allScanRangesLocations); + } + for (TScanRangeLocations onePartitionOneScanRangeLocation : allScanRangesLocations) { // usually, the onePartitionOneScanRangeLocation is a tablet in one partition long bytes = getScanRangeSize(scanNode, onePartitionOneScanRangeLocation); WorkerScanRanges assigned = selectScanReplicaAndMinWorkloadWorker( - onePartitionOneScanRangeLocation, bytes); + onePartitionOneScanRangeLocation, bytes, orderedScanRangeLocations); UninstancedScanSource scanRanges = workerScanRanges.computeIfAbsent( assigned.worker, w -> new UninstancedScanSource( @@ -99,22 +111,37 @@ public Map selectReplicaAndWorkerW @Override public Map selectReplicaAndWorkerWithBucket( - UnassignedScanBucketOlapTableJob unassignedJob) { + UnassignedScanBucketOlapTableJob unassignedJob, ConnectContext context) { PlanFragment fragment = unassignedJob.getFragment(); - List scanNodes = unassignedJob.getScanNodes(); + List scanNodes = unassignedJob.getScanNodes(); List olapScanNodes = unassignedJob.getOlapScanNodes(); - BiFunction> bucketScanRangeSupplier = bucketScanRangeSupplier(); + boolean orderedScanRangeLocations = shouldSortTablets(scanNodes, context); + if (orderedScanRangeLocations) { + List sortedOlapScanNodes = Lists.newArrayList(olapScanNodes); + sortedOlapScanNodes.sort(Comparator.comparing(node -> node.getId().asInt())); + scanNodes = olapScanNodes = sortedOlapScanNodes; + } + + BiFunction> bucketScanRangeSupplier + = bucketScanRangeSupplier(orderedScanRangeLocations); Function> bucketBytesSupplier = bucketBytesSupplier(); // all are olap scan nodes if (!scanNodes.isEmpty() && scanNodes.size() == olapScanNodes.size()) { if (olapScanNodes.size() == 1 && fragment.hasBucketShuffleJoin()) { - return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + return selectForBucket( + unassignedJob, scanNodes, bucketScanRangeSupplier, + bucketBytesSupplier, orderedScanRangeLocations + ); } else if (fragment.hasColocatePlanNode()) { - return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + return selectForBucket( + unassignedJob, scanNodes, bucketScanRangeSupplier, + bucketBytesSupplier, orderedScanRangeLocations + ); } } else if (olapScanNodes.isEmpty() && fragment.getDataPartition() == DataPartition.UNPARTITIONED) { - return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + return selectForBucket( + unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier, orderedScanRangeLocations); } throw new IllegalStateException( "Illegal bucket shuffle join or colocate join in fragment:\n" @@ -122,10 +149,16 @@ public Map selectReplicaAndWorkerW ); } - private BiFunction> bucketScanRangeSupplier() { + private BiFunction> bucketScanRangeSupplier( + boolean shouldSortTablets) { return (scanNode, bucketIndex) -> { if (scanNode instanceof OlapScanNode) { - return (List) ((OlapScanNode) scanNode).bucketSeq2locations.get(bucketIndex); + List scanRangeLocations + = ((OlapScanNode) scanNode).bucketSeq2locations.get(bucketIndex); + if (shouldSortTablets) { + scanRangeLocations = sortScanRanges(scanRangeLocations); + } + return scanRangeLocations; } else { // the backend is selected by XxxScanNode.createScanRangeLocations() return scanNode.getScanRangeLocations(0); @@ -145,9 +178,10 @@ private Function> bucketBytesSupplier() { } private Map selectForBucket( - UnassignedJob unassignedJob, List scanNodes, + UnassignedJob unassignedJob, List scanNodes, BiFunction> bucketScanRangeSupplier, - Function> bucketBytesSupplier) { + Function> bucketBytesSupplier, + boolean orderedScanRangeLocations) { Map assignment = Maps.newLinkedHashMap(); Map bucketIndexToBytes = computeEachBucketScanBytes(scanNodes, bucketBytesSupplier); @@ -162,7 +196,10 @@ private Map selectForBucket( = bucketScanRangeSupplier.apply(scanNode, bucketIndex); if (!allPartitionTabletsInOneBucketInOneTable.isEmpty()) { WorkerScanRanges replicaAndWorker = selectScanReplicaAndMinWorkloadWorker( - allPartitionTabletsInOneBucketInOneTable.get(0), allScanNodeScanBytesInOneBucket); + allPartitionTabletsInOneBucketInOneTable.get(0), + allScanNodeScanBytesInOneBucket, + orderedScanRangeLocations + ); selectedWorker = replicaAndWorker.worker; break; } @@ -199,8 +236,13 @@ private Map selectForBucket( } private WorkerScanRanges selectScanReplicaAndMinWorkloadWorker( - TScanRangeLocations tabletLocation, long tabletBytes) { + TScanRangeLocations tabletLocation, long tabletBytes, boolean orderedScanRangeLocations) { List replicaLocations = tabletLocation.getLocations(); + if (orderedScanRangeLocations) { + replicaLocations = Lists.newArrayList(replicaLocations); + Collections.sort(replicaLocations); + } + int replicaNum = replicaLocations.size(); WorkerWorkload minWorkload = new WorkerWorkload(Integer.MAX_VALUE, Long.MAX_VALUE); DistributedPlanWorker minWorkLoadWorker = null; @@ -269,7 +311,7 @@ private List> filterReplicaByWorkerInBucket( } private Map computeEachBucketScanBytes( - List scanNodes, Function> bucketBytesSupplier) { + List scanNodes, Function> bucketBytesSupplier) { Map bucketIndexToBytes = Maps.newLinkedHashMap(); for (ScanNode scanNode : scanNodes) { Map bucketSeq2Bytes = bucketBytesSupplier.apply(scanNode); @@ -299,8 +341,11 @@ private long getScanRangeSize(ScanNode scanNode, TScanRangeLocations scanRangeLo if (extScanRange != null) { TFileScanRange fileScanRange = extScanRange.getFileScanRange(); long size = 0; - for (TFileRangeDesc range : fileScanRange.getRanges()) { - size += range.getSize(); + List ranges = fileScanRange.getRanges(); + if (ranges != null) { + for (TFileRangeDesc range : ranges) { + size += range.getSize(); + } } return size; } @@ -308,6 +353,22 @@ private long getScanRangeSize(ScanNode scanNode, TScanRangeLocations scanRangeLo return 0L; } + private boolean shouldSortTablets(List scanNodes, ConnectContext context) { + return scanNodes.stream().allMatch(OlapScanNode.class::isInstance) + && context.getSessionVariable().enableOrderedScanRangeLocations; + } + + private List sortScanRanges(List tablets) { + tablets = Lists.newArrayList(tablets); + tablets.sort( + (p1, p2) -> org.apache.thrift.TBaseHelper.compareTo( + p1.getScanRange().getPaloScanRange().tablet_id, + p2.getScanRange().getPaloScanRange().tablet_id + ) + ); + return tablets; + } + private static class WorkerWorkload implements Comparable { private int taskNum; private long scanBytes; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java index 40876a09e44301..34e524dc8536f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedScanBucketOlapTableJob; import org.apache.doris.nereids.trees.plans.distribute.worker.job.UninstancedScanSource; import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.TScanRangeLocation; import org.apache.doris.thrift.TScanRangeLocations; import org.apache.doris.thrift.TScanRangeParams; @@ -35,7 +36,8 @@ public interface ScanWorkerSelector { // for a scan node, select replica for each scan range(denote tablet if the ScanNode is OlapScanNode), // use the replica location to build a worker execute the instance - Map selectReplicaAndWorkerWithoutBucket(ScanNode scanNode); + Map selectReplicaAndWorkerWithoutBucket( + ScanNode scanNode, ConnectContext context); // return // key: Worker, the backend which will process this fragment @@ -49,14 +51,12 @@ public interface ScanWorkerSelector { // and distributed by hash(id) buckets 10. And, so, there has 10 buckets from bucket 0 to // bucket 9, and every bucket contains two tablets, because there are two partitions. Map selectReplicaAndWorkerWithBucket( - UnassignedScanBucketOlapTableJob unassignedJob); + UnassignedScanBucketOlapTableJob unassignedJob, ConnectContext context); static TScanRangeParams buildScanReplicaParams( TScanRangeLocations tabletLocation, TScanRangeLocation replicaLocation) { TScanRangeParams replicaParam = new TScanRangeParams(); replicaParam.scan_range = tabletLocation.scan_range; - // Volume is optional, so we need to set the value and the is-set bit - replicaParam.setVolumeId(replicaLocation.volume_id); return replicaParam; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java index f53ee614523379..8003abf0e7dae2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.AbstractTreeNode; import org.apache.doris.nereids.util.Utils; import org.apache.doris.planner.ExchangeNode; @@ -31,14 +32,16 @@ /** AbstractUnassignedJob */ public abstract class AbstractUnassignedJob extends AbstractTreeNode implements UnassignedJob { + protected final StatementContext statementContext; protected final PlanFragment fragment; protected final List scanNodes; protected final ListMultimap exchangeToChildJob; /** AbstractUnassignedJob */ - public AbstractUnassignedJob(PlanFragment fragment, List scanNodes, - ListMultimap exchangeToChildJob) { + public AbstractUnassignedJob(StatementContext statementContext, PlanFragment fragment, + List scanNodes, ListMultimap exchangeToChildJob) { super(Utils.fastToImmutableList(exchangeToChildJob.values())); + this.statementContext = Objects.requireNonNull(statementContext, "statementContext can not be null"); this.fragment = Objects.requireNonNull(fragment, "fragment can not be null"); this.scanNodes = Utils.fastToImmutableList( Objects.requireNonNull(scanNodes, "scanNodes can not be null") @@ -47,6 +50,11 @@ public AbstractUnassignedJob(PlanFragment fragment, List scanNodes, = Objects.requireNonNull(exchangeToChildJob, "exchangeToChildJob can not be null"); } + @Override + public StatementContext getStatementContext() { + return statementContext; + } + @Override public PlanFragment getFragment() { return fragment; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java index 2557966b6e84a8..1ea53a85bad2a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.planner.ExchangeNode; @@ -39,17 +40,17 @@ public abstract class AbstractUnassignedScanJob extends AbstractUnassignedJob { protected final AtomicInteger shareScanIdGenerator = new AtomicInteger(); - public AbstractUnassignedScanJob(PlanFragment fragment, + public AbstractUnassignedScanJob(StatementContext statementContext, PlanFragment fragment, List scanNodes, ListMultimap exchangeToChildJob) { - super(fragment, scanNodes, exchangeToChildJob); + super(statementContext, fragment, scanNodes, exchangeToChildJob); } @Override - public List computeAssignedJobs(DistributedPlanWorkerManager workerManager, - ListMultimap inputJobs) { + public List computeAssignedJobs( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { - Map workerToScanSource = multipleMachinesParallelization( - workerManager, inputJobs); + Map workerToScanSource + = multipleMachinesParallelization(workerManager, inputJobs); List assignedJobs = insideMachineParallelization(workerToScanSource, inputJobs, workerManager); @@ -70,7 +71,7 @@ protected List insideMachineParallelization( Map workerToScanRanges, ListMultimap inputJobs, DistributedPlanWorkerManager workerManager) { - ConnectContext context = ConnectContext.get(); + ConnectContext context = statementContext.getConnectContext(); boolean useLocalShuffleToAddParallel = useLocalShuffleToAddParallel(workerToScanRanges); int instanceIndexInFragment = 0; List instances = Lists.newArrayList(); @@ -98,27 +99,31 @@ protected List insideMachineParallelization( scanNodes, 1 ); - // Some tablets too big, we need add parallel to process these tablets after scan, - // for example, use one OlapScanNode to scan data, and use some local instances - // to process Aggregation parallel. We call it `share scan`. Backend will know this - // instances share the same ScanSource, and will not scan same data multiple times. + // when data not big, but aggregation too slow, we will use 1 instance to scan data, + // and use more instances (to ***add parallel***) to process aggregate. + // We call it `ignore data distribution` of `share scan`. Backend will know this instances + // share the same ScanSource, and will not scan same data multiple times. // // +-------------------------------- same fragment in one host -------------------------------------+ // | instance1 instance2 instance3 instance4 | // | \ \ / / | // | | // | OlapScanNode | - // |(share scan node, and local shuffle data to other local instances to parallel compute this data)| + // |(share scan node, instance1 will scan all data and local shuffle to other local instances | + // | to parallel compute this data) | // +------------------------------------------------------------------------------------------------+ ScanSource shareScanSource = instanceToScanRanges.get(0); // one scan range generate multiple instances, // different instances reference the same scan source int shareScanId = shareScanIdGenerator.getAndIncrement(); + ScanSource emptyShareScanSource = shareScanSource.newEmpty(); for (int i = 0; i < instanceNum; i++) { LocalShuffleAssignedJob instance = new LocalShuffleAssignedJob( - instanceIndexInFragment++, shareScanId, context.nextInstanceId(), - this, worker, shareScanSource); + instanceIndexInFragment++, shareScanId, i > 0, + context.nextInstanceId(), this, worker, + i == 0 ? shareScanSource : emptyShareScanSource + ); instances.add(instance); } } else { @@ -147,6 +152,12 @@ protected List insideMachineParallelization( protected boolean useLocalShuffleToAddParallel( Map workerToScanRanges) { + if (fragment.queryCacheParam != null) { + return false; + } + if (fragment.hasNullAwareLeftAntiJoin()) { + return false; + } if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isForceToLocalShuffle()) { return true; } @@ -181,6 +192,10 @@ protected int degreeOfParallelism(int maxParallel) { if (!fragment.getDataPartition().isPartitioned()) { return 1; } + if (fragment.queryCacheParam != null) { + // backend need use one instance for one tablet to look up tablet query cache + return maxParallel; + } if (scanNodes.size() == 1 && scanNodes.get(0) instanceof OlapScanNode) { OlapScanNode olapScanNode = (OlapScanNode) scanNodes.get(0); // if the scan node have limit and no conjuncts, only need 1 instance to save cpu and mem resource, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java index f9f6b9dea1451b..d74626b4889468 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java @@ -36,4 +36,5 @@ public interface AssignedJob { ScanSource getScanSource(); String toString(boolean showUnassignedJob); + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java index 33d066a02b9fcb..c9fb78db8255c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java @@ -36,7 +36,7 @@ public class BucketScanSource extends ScanSource { public final Map> bucketIndexToScanNodeToTablets; public BucketScanSource(Map> bucketIndexToScanNodeToTablets) { - this.bucketIndexToScanNodeToTablets = bucketIndexToScanNodeToTablets; + this.bucketIndexToScanNodeToTablets = Maps.newLinkedHashMap(bucketIndexToScanNodeToTablets); } @Override @@ -141,6 +141,11 @@ public void toString(StringBuilder str, String prefix) { str.append("\n").append(prefix).append("]"); } + @Override + public ScanSource newEmpty() { + return new BucketScanSource(Maps.newLinkedHashMap()); + } + @Override public boolean isEmpty() { return bucketIndexToScanNodeToTablets.isEmpty(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java index 929f70e73a7e4d..1aa3c3361bee6d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import java.util.List; import java.util.Map; @@ -35,7 +36,7 @@ public class DefaultScanSource extends ScanSource { public final Map scanNodeToScanRanges; public DefaultScanSource(Map scanNodeToScanRanges) { - this.scanNodeToScanRanges = scanNodeToScanRanges; + this.scanNodeToScanRanges = Maps.newLinkedHashMap(scanNodeToScanRanges); } public static DefaultScanSource empty() { @@ -82,11 +83,6 @@ public boolean isEmpty() { return scanNodeToScanRanges.isEmpty(); } - @Override - public void toString(StringBuilder str, String prefix) { - toString(scanNodeToScanRanges, str, prefix); - } - /** toString */ public static void toString(Map scanNodeToScanRanges, StringBuilder str, String prefix) { if (scanNodeToScanRanges.isEmpty()) { @@ -112,4 +108,14 @@ public static void toString(Map scanNodeToScanRanges, Stri } str.append("\n").append(prefix).append("]"); } + + @Override + public void toString(StringBuilder str, String prefix) { + toString(scanNodeToScanRanges, str, prefix); + } + + @Override + public ScanSource newEmpty() { + return empty(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java index 50e43fc0282755..2ba269a5a7b89f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java @@ -24,20 +24,35 @@ import java.util.Map; -/** LocalShuffleAssignedJob */ +/** + * LocalShuffleAssignedJob: + * this instance will use ignore_data_distribution function of local shuffle to add parallel + * after scan data + */ public class LocalShuffleAssignedJob extends StaticAssignedJob { public final int shareScanId; + public final boolean receiveDataFromLocal; public LocalShuffleAssignedJob( - int indexInUnassignedJob, int shareScanId, TUniqueId instanceId, + int indexInUnassignedJob, int shareScanId, boolean receiveDataFromLocal, TUniqueId instanceId, UnassignedJob unassignedJob, DistributedPlanWorker worker, ScanSource scanSource) { super(indexInUnassignedJob, instanceId, unassignedJob, worker, scanSource); this.shareScanId = shareScanId; + this.receiveDataFromLocal = receiveDataFromLocal; } @Override protected Map extraInfo() { return ImmutableMap.of("shareScanIndex", String.valueOf(shareScanId)); } + + @Override + protected String formatScanSourceString() { + if (receiveDataFromLocal) { + return "read data from first instance of " + getAssignedWorker(); + } else { + return super.formatScanSourceString(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java index b124e14bd73c77..5802f8ab7efd85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java @@ -30,6 +30,8 @@ public abstract class ScanSource { public abstract boolean isEmpty(); + public abstract ScanSource newEmpty(); + public abstract void toString(StringBuilder str, String prefix); @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java index 1a92cf71019e66..75849ad8146737 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java @@ -77,12 +77,6 @@ public String toString() { @Override public String toString(boolean showUnassignedJob) { - StringBuilder scanSourceString = new StringBuilder(); - if (!scanSource.isEmpty()) { - scanSource.toString(scanSourceString, " "); - } else { - scanSourceString = new StringBuilder("[]"); - } StringBuilder str = new StringBuilder(getClass().getSimpleName()).append("("); if (showUnassignedJob) { str.append("\n unassignedJob: ").append(unassignedJob).append(","); @@ -95,7 +89,7 @@ public String toString(boolean showUnassignedJob) { } return str - .append(",\n scanSource: " + scanSourceString) + .append(",\n scanSource: " + formatScanSourceString()) .append("\n)") .toString(); } @@ -103,4 +97,19 @@ public String toString(boolean showUnassignedJob) { protected Map extraInfo() { return ImmutableMap.of(); } + + protected String formatScanSourceString() { + StringBuilder scanSourceString = new StringBuilder(); + if (!scanSource.isEmpty()) { + scanSource.toString(scanSourceString, " "); + } else { + scanSourceString = new StringBuilder("[]"); + } + return scanSourceString.toString(); + } + + @Override + public int hashCode() { + return indexInUnassignedJob; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java index 558af8844974fc..72feb5c7b1a94b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -38,9 +39,9 @@ /** UnassignedGatherScanMultiRemoteTablesJob */ public class UnassignedGatherScanMultiRemoteTablesJob extends AbstractUnassignedJob { - public UnassignedGatherScanMultiRemoteTablesJob(PlanFragment fragment, + public UnassignedGatherScanMultiRemoteTablesJob(StatementContext statementContext, PlanFragment fragment, List scanNodes, ListMultimap exchangeToChildJob) { - super(fragment, scanNodes, exchangeToChildJob); + super(statementContext, fragment, scanNodes, exchangeToChildJob); } /** canApply */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGroupCommitJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGroupCommitJob.java new file mode 100644 index 00000000000000..7371b9ed2f9203 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGroupCommitJob.java @@ -0,0 +1,52 @@ +// 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.distribute.worker.job; + +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.trees.plans.distribute.worker.BackendWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ListMultimap; + +import java.util.List; + +/** UnassignedGroupCommitJob */ +public class UnassignedGroupCommitJob extends AbstractUnassignedJob { + public UnassignedGroupCommitJob(StatementContext statementContext, + PlanFragment fragment, List scanNodes, + ListMultimap exchangeToChildJob) { + super(statementContext, fragment, scanNodes, exchangeToChildJob); + } + + @Override + public List computeAssignedJobs( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { + TUniqueId instanceId = statementContext.getConnectContext().nextInstanceId(); + BackendWorker selectBackend = new BackendWorker(statementContext.getGroupCommitMergeBackend()); + return ImmutableList.of( + new StaticAssignedJob( + 0, instanceId, this, selectBackend, DefaultScanSource.empty() + ) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java index d142460ea2db02..51ab08280b33cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.TreeNode; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; @@ -34,6 +35,8 @@ * for example: a fragment job, which doesn't parallelization to some instance jobs and also no worker to invoke it */ public interface UnassignedJob extends TreeNode { + StatementContext getStatementContext(); + PlanFragment getFragment(); List getScanNodes(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java index 396ba51e01b4ee..8421bc2442cf2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping; import org.apache.doris.nereids.trees.plans.distribute.worker.LoadBalanceScanWorkerSelector; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -36,6 +37,7 @@ import com.google.common.collect.ListMultimap; import com.google.common.collect.Maps; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -50,39 +52,49 @@ public class UnassignedJobBuilder { /** * build job from fragment. */ - public static FragmentIdMapping buildJobs(FragmentIdMapping fragments) { + public static FragmentIdMapping buildJobs( + StatementContext statementContext, FragmentIdMapping fragments) { UnassignedJobBuilder builder = new UnassignedJobBuilder(); FragmentLineage fragmentLineage = buildFragmentLineage(fragments); FragmentIdMapping unassignedJobs = new FragmentIdMapping<>(); // build from leaf to parent - for (Entry kv : fragments.entrySet()) { + Iterator> iterator = fragments.entrySet().iterator(); + while (iterator.hasNext()) { + Entry kv = iterator.next(); + boolean isTopFragment = !iterator.hasNext(); + PlanFragmentId fragmentId = kv.getKey(); PlanFragment fragment = kv.getValue(); ListMultimap inputJobs = findInputJobs( fragmentLineage, fragmentId, unassignedJobs); - UnassignedJob unassignedJob = builder.buildJob(fragment, inputJobs); + UnassignedJob unassignedJob = builder.buildJob(statementContext, fragment, inputJobs, isTopFragment); unassignedJobs.put(fragmentId, unassignedJob); } + return unassignedJobs; } private UnassignedJob buildJob( - PlanFragment planFragment, ListMultimap inputJobs) { + StatementContext statementContext, PlanFragment planFragment, + ListMultimap inputJobs, boolean isTopFragment) { List scanNodes = collectScanNodesInThisFragment(planFragment); if (planFragment.specifyInstances.isPresent()) { - return buildSpecifyInstancesJob(planFragment, scanNodes, inputJobs); + return buildSpecifyInstancesJob(statementContext, planFragment, scanNodes, inputJobs); + } else if (scanNodes.isEmpty() && isTopFragment + && statementContext.getGroupCommitMergeBackend() != null) { + return new UnassignedGroupCommitJob(statementContext, planFragment, scanNodes, inputJobs); } else if (!scanNodes.isEmpty() || isLeafFragment(planFragment)) { - return buildLeafOrScanJob(planFragment, scanNodes, inputJobs); + return buildLeafOrScanJob(statementContext, planFragment, scanNodes, inputJobs); } else { - return buildShuffleJob(planFragment, inputJobs); + return buildShuffleJob(statementContext, planFragment, inputJobs); } } private UnassignedJob buildLeafOrScanJob( - PlanFragment planFragment, List scanNodes, + StatementContext statementContext, PlanFragment planFragment, List scanNodes, ListMultimap inputJobs) { int olapScanNodeNum = olapScanNodeNum(scanNodes); @@ -91,20 +103,26 @@ private UnassignedJob buildLeafOrScanJob( // we need assign a backend which contains the data, // so that the OlapScanNode can find the data in the backend // e.g. select * from olap_table - unassignedJob = buildScanOlapTableJob(planFragment, (List) scanNodes, inputJobs, scanWorkerSelector); + unassignedJob = buildScanOlapTableJob( + statementContext, planFragment, (List) scanNodes, inputJobs, scanWorkerSelector + ); } else if (scanNodes.isEmpty()) { // select constant without table, // e.g. select 100 union select 200 - unassignedJob = buildQueryConstantJob(planFragment); + unassignedJob = buildQueryConstantJob(statementContext, planFragment); } else if (olapScanNodeNum == 0) { ScanNode scanNode = scanNodes.get(0); if (scanNode instanceof SchemaScanNode) { // select * from information_schema.tables - unassignedJob = buildScanMetadataJob(planFragment, (SchemaScanNode) scanNode, scanWorkerSelector); + unassignedJob = buildScanMetadataJob( + statementContext, planFragment, (SchemaScanNode) scanNode, scanWorkerSelector + ); } else { // only scan external tables or cloud tables or table valued functions // e,g. select * from numbers('number'='100') - unassignedJob = buildScanRemoteTableJob(planFragment, scanNodes, inputJobs, scanWorkerSelector); + unassignedJob = buildScanRemoteTableJob( + statementContext, planFragment, scanNodes, inputJobs, scanWorkerSelector + ); } } @@ -117,20 +135,21 @@ private UnassignedJob buildLeafOrScanJob( } private UnassignedJob buildSpecifyInstancesJob( - PlanFragment planFragment, List scanNodes, ListMultimap inputJobs) { - return new UnassignedSpecifyInstancesJob(planFragment, scanNodes, inputJobs); + StatementContext statementContext, PlanFragment planFragment, + List scanNodes, ListMultimap inputJobs) { + return new UnassignedSpecifyInstancesJob(statementContext, planFragment, scanNodes, inputJobs); } private UnassignedJob buildScanOlapTableJob( - PlanFragment planFragment, List olapScanNodes, + StatementContext statementContext, PlanFragment planFragment, List olapScanNodes, ListMultimap inputJobs, ScanWorkerSelector scanWorkerSelector) { if (shouldAssignByBucket(planFragment)) { return new UnassignedScanBucketOlapTableJob( - planFragment, olapScanNodes, inputJobs, scanWorkerSelector); + statementContext, planFragment, olapScanNodes, inputJobs, scanWorkerSelector); } else if (olapScanNodes.size() == 1) { return new UnassignedScanSingleOlapTableJob( - planFragment, olapScanNodes.get(0), inputJobs, scanWorkerSelector); + statementContext, planFragment, olapScanNodes.get(0), inputJobs, scanWorkerSelector); } else { throw new IllegalStateException("Not supported multiple scan multiple " + "OlapTable but not contains colocate join or bucket shuffle join: " @@ -156,34 +175,37 @@ private boolean isLeafFragment(PlanFragment planFragment) { return planFragment.getChildren().isEmpty(); } - private UnassignedQueryConstantJob buildQueryConstantJob(PlanFragment planFragment) { - return new UnassignedQueryConstantJob(planFragment); + private UnassignedQueryConstantJob buildQueryConstantJob( + StatementContext statementContext, PlanFragment planFragment) { + return new UnassignedQueryConstantJob(statementContext, planFragment); } private UnassignedJob buildScanMetadataJob( - PlanFragment fragment, SchemaScanNode schemaScanNode, ScanWorkerSelector scanWorkerSelector) { - return new UnassignedScanMetadataJob(fragment, schemaScanNode, scanWorkerSelector); + StatementContext statementContext, PlanFragment fragment, + SchemaScanNode schemaScanNode, ScanWorkerSelector scanWorkerSelector) { + return new UnassignedScanMetadataJob(statementContext, fragment, schemaScanNode, scanWorkerSelector); } private UnassignedJob buildScanRemoteTableJob( - PlanFragment planFragment, List scanNodes, + StatementContext statementContext, PlanFragment planFragment, List scanNodes, ListMultimap inputJobs, ScanWorkerSelector scanWorkerSelector) { if (scanNodes.size() == 1) { return new UnassignedScanSingleRemoteTableJob( - planFragment, scanNodes.get(0), inputJobs, scanWorkerSelector); + statementContext, planFragment, scanNodes.get(0), inputJobs, scanWorkerSelector); } else if (UnassignedGatherScanMultiRemoteTablesJob.canApply(scanNodes)) { // select * from numbers("number" = "10") a union all select * from numbers("number" = "20") b; // use an instance to scan table a and table b - return new UnassignedGatherScanMultiRemoteTablesJob(planFragment, scanNodes, inputJobs); + return new UnassignedGatherScanMultiRemoteTablesJob(statementContext, planFragment, scanNodes, inputJobs); } else { return null; } } private UnassignedShuffleJob buildShuffleJob( - PlanFragment planFragment, ListMultimap inputJobs) { - return new UnassignedShuffleJob(planFragment, inputJobs); + StatementContext statementContext, PlanFragment planFragment, + ListMultimap inputJobs) { + return new UnassignedShuffleJob(statementContext, planFragment, inputJobs); } private static ListMultimap findInputJobs( @@ -246,16 +268,12 @@ private static boolean shouldAssignByBucket(PlanFragment fragment) { if (fragment.hasColocatePlanNode()) { return true; } - if (enableBucketShuffleJoin() && fragment.hasBucketShuffleJoin()) { + if (fragment.hasBucketShuffleJoin()) { return true; } return false; } - private static boolean enableBucketShuffleJoin() { - return true; - } - // the class support find exchange nodes in the fragment, and find child fragment by exchange node id private static class FragmentLineage { private final FragmentIdMapping> parentFragmentToExchangeNode; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java index bfbafd0739065c..b27ae4c60f97b7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.planner.ExchangeNode; @@ -32,8 +33,8 @@ /** UnassignedQueryConstantJob */ public class UnassignedQueryConstantJob extends AbstractUnassignedJob { - public UnassignedQueryConstantJob(PlanFragment fragment) { - super(fragment, ImmutableList.of(), ArrayListMultimap.create()); + public UnassignedQueryConstantJob(StatementContext statementContext, PlanFragment fragment) { + super(statementContext, fragment, ImmutableList.of(), ArrayListMultimap.create()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java index 2003815e8d7c14..f528dfb110bf82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Tablet; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -58,10 +59,10 @@ public class UnassignedScanBucketOlapTableJob extends AbstractUnassignedScanJob /** UnassignedScanNativeTableJob */ public UnassignedScanBucketOlapTableJob( - PlanFragment fragment, List olapScanNodes, + StatementContext statementContext, PlanFragment fragment, List olapScanNodes, ListMultimap exchangeToChildJob, ScanWorkerSelector scanWorkerSelector) { - super(fragment, (List) olapScanNodes, exchangeToChildJob); + super(statementContext, fragment, (List) olapScanNodes, exchangeToChildJob); this.scanWorkerSelector = Objects.requireNonNull( scanWorkerSelector, "scanWorkerSelector cat not be null"); @@ -94,7 +95,9 @@ protected Map multipleMachinesPara // ... // } // } - return scanWorkerSelector.selectReplicaAndWorkerWithBucket(this); + return scanWorkerSelector.selectReplicaAndWorkerWithBucket( + this, statementContext.getConnectContext() + ); } @Override @@ -219,7 +222,7 @@ private List fillUpInstances(List instances) { if (!mergedBucketsInSameWorkerInstance) { fillUpInstance = new LocalShuffleAssignedJob( newInstances.size(), shareScanIdGenerator.getAndIncrement(), - context.nextInstanceId(), this, worker, scanSource + false, context.nextInstanceId(), this, worker, scanSource ); } } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanMetadataJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanMetadataJob.java index d15bed010619b1..b76d7628f86094 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanMetadataJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanMetadataJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -37,9 +38,10 @@ public class UnassignedScanMetadataJob extends AbstractUnassignedScanJob { private final SchemaScanNode schemaScanNode; private final ScanWorkerSelector scanWorkerSelector; - public UnassignedScanMetadataJob(PlanFragment fragment, SchemaScanNode schemaScanNode, - ScanWorkerSelector scanWorkerSelector) { - super(fragment, ImmutableList.of(schemaScanNode), ArrayListMultimap.create()); + public UnassignedScanMetadataJob( + StatementContext statementContext, PlanFragment fragment, + SchemaScanNode schemaScanNode, ScanWorkerSelector scanWorkerSelector) { + super(statementContext, fragment, ImmutableList.of(schemaScanNode), ArrayListMultimap.create()); this.scanWorkerSelector = Objects.requireNonNull( scanWorkerSelector, "scanWorkerSelector cat not be null"); this.schemaScanNode = schemaScanNode; @@ -48,7 +50,9 @@ public UnassignedScanMetadataJob(PlanFragment fragment, SchemaScanNode schemaSca @Override protected Map multipleMachinesParallelization( DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { - return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket(schemaScanNode); + return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket( + schemaScanNode, statementContext.getConnectContext() + ); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java index ecbadab8af3a93..8f6e56cb9bc7fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -37,10 +38,10 @@ public class UnassignedScanSingleOlapTableJob extends AbstractUnassignedScanJob private final ScanWorkerSelector scanWorkerSelector; public UnassignedScanSingleOlapTableJob( - PlanFragment fragment, OlapScanNode olapScanNode, + StatementContext statementContext, PlanFragment fragment, OlapScanNode olapScanNode, ListMultimap exchangeToChildJob, ScanWorkerSelector scanWorkerSelector) { - super(fragment, ImmutableList.of(olapScanNode), exchangeToChildJob); + super(statementContext, fragment, ImmutableList.of(olapScanNode), exchangeToChildJob); this.scanWorkerSelector = Objects.requireNonNull( scanWorkerSelector, "scanWorkerSelector cat not be null"); this.olapScanNode = olapScanNode; @@ -57,7 +58,9 @@ protected Map multipleMachinesPara // BackendWorker("172.0.0.2"): // olapScanNode1: ScanRanges([tablet_10005, tablet_10006, tablet_10007, tablet_10008, tablet_10009]) // } - return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket(olapScanNode); + return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket( + olapScanNode, statementContext.getConnectContext() + ); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java index 08e48202fcf90f..560b032199ad49 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; @@ -40,16 +41,18 @@ public class UnassignedScanSingleRemoteTableJob extends AbstractUnassignedScanJo private final ScanWorkerSelector scanWorkerSelector; public UnassignedScanSingleRemoteTableJob( - PlanFragment fragment, ScanNode scanNode, ListMultimap exchangeToChildJob, - ScanWorkerSelector scanWorkerSelector) { - super(fragment, ImmutableList.of(scanNode), exchangeToChildJob); + StatementContext statementContext, PlanFragment fragment, ScanNode scanNode, + ListMultimap exchangeToChildJob, ScanWorkerSelector scanWorkerSelector) { + super(statementContext, fragment, ImmutableList.of(scanNode), exchangeToChildJob); this.scanWorkerSelector = Objects.requireNonNull(scanWorkerSelector, "scanWorkerSelector is not null"); } @Override protected Map multipleMachinesParallelization( DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { - return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket(scanNodes.get(0)); + return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket( + scanNodes.get(0), statementContext.getConnectContext() + ); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java index 5b1cb1f9878fb1..e2793119c2c505 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java @@ -17,10 +17,13 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.planner.ExchangeNode; import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.PlanNode; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; @@ -38,8 +41,10 @@ /** UnassignedShuffleJob */ public class UnassignedShuffleJob extends AbstractUnassignedJob { - public UnassignedShuffleJob(PlanFragment fragment, ListMultimap exchangeToChildJob) { - super(fragment, ImmutableList.of(), exchangeToChildJob); + public UnassignedShuffleJob( + StatementContext statementContext, PlanFragment fragment, + ListMultimap exchangeToChildJob) { + super(statementContext, fragment, ImmutableList.of(), exchangeToChildJob); } @Override @@ -81,6 +86,13 @@ protected int degreeOfParallelism() { if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable() != null) { expectInstanceNum = ConnectContext.get().getSessionVariable().getExchangeInstanceParallel(); } + + // TODO: check nested loop join do right outer / semi / anti join + PlanNode leftMostNode = findLeftmostNode(fragment.getPlanRoot()).second; + // when we use nested loop join do right outer / semi / anti join, the instance must be 1. + if (leftMostNode.getNumInstances() == 1) { + expectInstanceNum = 1; + } return expectInstanceNum; } @@ -121,4 +133,16 @@ private List distinctShuffleWorkers(List ins Collections.shuffle(candidateWorkers); return candidateWorkers; } + + // Returns the id of the leftmost node of any of the gives types in 'plan_root', + // or INVALID_PLAN_NODE_ID if no such node present. + private Pair findLeftmostNode(PlanNode plan) { + PlanNode childPlan = plan; + PlanNode fatherPlan = null; + while (childPlan.getChildren().size() != 0 && !(childPlan instanceof ExchangeNode)) { + fatherPlan = childPlan; + childPlan = childPlan.getChild(0); + } + return Pair.of(fatherPlan, childPlan); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java index c450161ae41905..c2775d4e8a1029 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.distribute.worker.job; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.distribute.NereidsSpecifyInstances; import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; import org.apache.doris.planner.ExchangeNode; @@ -33,9 +34,9 @@ public class UnassignedSpecifyInstancesJob extends AbstractUnassignedJob { private final NereidsSpecifyInstances specifyInstances; public UnassignedSpecifyInstancesJob( - PlanFragment fragment, List scanNodes, + StatementContext statementContext, PlanFragment fragment, List scanNodes, ListMultimap exchangeToChildJob) { - super(fragment, scanNodes, exchangeToChildJob); + super(statementContext, fragment, scanNodes, exchangeToChildJob); Preconditions.checkArgument(fragment.specifyInstances.isPresent(), "Missing fragment specifyInstances"); this.specifyInstances = fragment.specifyInstances.get(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java index 7af092871915e5..56ef6b78cddd33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java @@ -79,6 +79,10 @@ public ExchangeNode(PlanNodeId id, PlanNode inputNode) { computeTupleIds(); } + public TPartitionType getPartitionType() { + return partitionType; + } + public void setPartitionType(TPartitionType partitionType) { this.partitionType = partitionType; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java index 9c54bdb406cb5e..02633dc519ca78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java @@ -40,6 +40,9 @@ public void addToDest(ExchangeNode exchangeNode) { destNodeList.add(exchangeNode); } + public List getDestNodeList() { + return destNodeList; + } public List getDestFragmentList() { return destNodeList.stream().map(PlanNode::getFragment).collect(Collectors.toList()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java index 3e3c49bf67524a..2584c909d427ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java @@ -397,6 +397,10 @@ public PlanFragmentId getId() { return fragmentId; } + public ExchangeNode getDestNode() { + return destNode; + } + public PlanFragment getDestFragment() { if (destNode == null) { return null; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 4eda6775b5c51d..e5dafd835155c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -171,20 +171,20 @@ public class Coordinator implements CoordInterface { private static final Logger LOG = LogManager.getLogger(Coordinator.class); - private static final String localIP = FrontendOptions.getLocalHostAddress(); + public static final String localIP = FrontendOptions.getLocalHostAddress(); // Random is used to shuffle instances of partitioned private static final Random instanceRandom = new SecureRandom(); - private static ExecutorService backendRpcCallbackExecutor = ThreadPoolManager.newDaemonProfileThreadPool(32, 100, - "backend-rpc-callback", true); + public static ExecutorService backendRpcCallbackExecutor = ThreadPoolManager.newDaemonProfileThreadPool( + 32, 100, "backend-rpc-callback", true); // Overall status of the entire query; set to the first reported fragment error // status or to CANCELLED, if Cancel() is called. - Status queryStatus = new Status(); + private Status queryStatus = new Status(); // save of related backends of this query - Map addressToBackendID = Maps.newHashMap(); + private Map addressToBackendID = Maps.newHashMap(); protected ImmutableMap idToBackend = ImmutableMap.of(); @@ -193,7 +193,7 @@ public class Coordinator implements CoordInterface { private FragmentIdMapping distributedPlans; // scan node id -> TFileScanRangeParams - protected Map fileScanRangeParamsMap = Maps.newHashMap(); + private Map fileScanRangeParamsMap = Maps.newHashMap(); // Why do we use query global? // When `NOW()` function is in sql, we need only one now(), @@ -217,7 +217,7 @@ public class Coordinator implements CoordInterface { private boolean returnedAllResults = false; // populated in computeFragmentExecParams() - protected final Map fragmentExecParamsMap = Maps.newHashMap(); + private final Map fragmentExecParamsMap = Maps.newHashMap(); private final List fragments; @@ -226,7 +226,7 @@ public class Coordinator implements CoordInterface { private final Map, PipelineExecContext> pipelineExecContexts = new HashMap<>(); private final List needCheckPipelineExecContexts = Lists.newArrayList(); private List receivers = Lists.newArrayList(); - protected final List scanNodes; + private final List scanNodes; private int scanRangeNum = 0; // number of instances of this query, equals to // number of backends executing plan fragments on behalf of this query; @@ -266,16 +266,16 @@ public class Coordinator implements CoordInterface { private Backend groupCommitBackend; // Runtime filter merge instance address and ID - public TNetworkAddress runtimeFilterMergeAddr; - public TUniqueId runtimeFilterMergeInstanceId; + private TNetworkAddress runtimeFilterMergeAddr; + private TUniqueId runtimeFilterMergeInstanceId; // Runtime filter ID to the target instance address of the fragment, // that is expected to use this runtime filter, the instance address is not repeated - public Map> ridToTargetParam = Maps.newHashMap(); + private Map> ridToTargetParam = Maps.newHashMap(); // The runtime filter that expects the instance to be used - public List assignedRuntimeFilters = new ArrayList<>(); - public List topnFilters = new ArrayList<>(); + private List assignedRuntimeFilters = new ArrayList<>(); + private List topnFilters = new ArrayList<>(); // Runtime filter ID to the builder instance number - public Map ridToBuilderNum = Maps.newHashMap(); + private Map ridToBuilderNum = Maps.newHashMap(); private ConnectContext context; private StatsErrorEstimator statsErrorEstimator; @@ -303,7 +303,7 @@ public long getNumReceivedRows() { return numReceivedRows; } - public List gettWorkloadGroups() { + public List getTWorkloadGroups() { return tWorkloadGroups; } @@ -703,7 +703,7 @@ public void close() { } } - private void execInternal() throws Exception { + protected void execInternal() throws Exception { if (LOG.isDebugEnabled() && !scanNodes.isEmpty()) { LOG.debug("debug: in Coordinator::exec. query id: {}, planNode: {}", DebugUtil.printId(queryId), scanNodes.get(0).treeToThrift()); @@ -775,8 +775,7 @@ private void execInternal() throws Exception { deltaUrls = Lists.newArrayList(); loadCounters = Maps.newHashMap(); List relatedBackendIds = Lists.newArrayList(addressToBackendID.values()); - Env.getCurrentEnv().getLoadManager().initJobProgress(jobId, queryId, instanceIds, - relatedBackendIds); + Env.getCurrentEnv().getLoadManager().initJobProgress(jobId, queryId, instanceIds, relatedBackendIds); Env.getCurrentEnv().getProgressManager().addTotalScanNums(String.valueOf(jobId), scanRangeNum); LOG.info("dispatch load job: {} to {}", DebugUtil.printId(queryId), addressToBackendID.keySet()); } @@ -785,7 +784,7 @@ private void execInternal() throws Exception { sendPipelineCtx(); } - private void sendPipelineCtx() throws TException, RpcException, UserException { + protected void sendPipelineCtx() throws TException, RpcException, UserException { lock(); try { Multiset hostCounter = HashMultiset.create(); @@ -831,7 +830,7 @@ private void sendPipelineCtx() throws TException, RpcException, UserException { Long backendId = this.addressToBackendID.get(entry.getKey()); backendFragments.add(Pair.of(fragment.getFragmentId(), backendId)); PipelineExecContext pipelineExecContext = new PipelineExecContext(fragment.getFragmentId(), - entry.getValue(), backendId, executionProfile); + entry.getValue(), idToBackend.get(backendId), executionProfile, jobId); // Each tParam will set the total number of Fragments that need to be executed on the same BE, // and the BE will determine whether all Fragments have been executed based on this information. // Notice. load fragment has a small probability that FragmentNumOnHost is 0, for unknown reasons. @@ -851,7 +850,7 @@ private void sendPipelineCtx() throws TException, RpcException, UserException { PipelineExecContexts ctxs = beToPipelineExecCtxs.get(pipelineExecContext.backend.getId()); if (ctxs == null) { - ctxs = new PipelineExecContexts(pipelineExecContext.backend.getId(), + ctxs = new PipelineExecContexts(queryId, pipelineExecContext.backend, pipelineExecContext.brpcAddress, twoPhaseExecution, entry.getValue().getFragmentNumOnHost()); beToPipelineExecCtxs.putIfAbsent(pipelineExecContext.backend.getId(), ctxs); @@ -946,7 +945,7 @@ private void sendPipelineCtx() throws TException, RpcException, UserException { } } - private Map> waitPipelineRpc(List> waitPipelineRpc(List>>> futures, long leftTimeMs, String operation) throws RpcException, UserException { if (leftTimeMs <= 0) { @@ -1326,7 +1325,7 @@ private void cancelLatch() { } } - private void cancelInternal(Status cancelReason) { + protected void cancelInternal(Status cancelReason) { for (ResultReceiver receiver : receivers) { receiver.cancel(cancelReason); } @@ -2834,7 +2833,7 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc protected final BucketShuffleJoinController bucketShuffleJoinController = new BucketShuffleJoinController(fragmentIdToScanNodeIds); - public class PipelineExecContext { + public static class PipelineExecContext { TPipelineFragmentParams rpcParams; PlanFragmentId fragmentId; boolean initiated; @@ -2845,23 +2844,25 @@ public class PipelineExecContext { Backend backend; long lastMissingHeartbeatTime = -1; long beProcessEpoch = 0; + private long jobId; public PipelineExecContext(PlanFragmentId fragmentId, - TPipelineFragmentParams rpcParams, Long backendId, - ExecutionProfile executionProfile) { + TPipelineFragmentParams rpcParams, Backend backend, + ExecutionProfile executionProfile, long jobId) { this.fragmentId = fragmentId; this.rpcParams = rpcParams; this.initiated = false; this.done = false; - this.backend = idToBackend.get(backendId); + this.backend = backend; this.address = new TNetworkAddress(backend.getHost(), backend.getBePort()); this.brpcAddress = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); this.beProcessEpoch = backend.getProcessEpoch(); + this.jobId = jobId; this.lastMissingHeartbeatTime = backend.getLastMissingHeartbeatTime(); - executionProfile.addFragmentBackend(fragmentId, backendId); + executionProfile.addFragmentBackend(fragmentId, backend.getId()); } /** @@ -2913,8 +2914,10 @@ public List buildFragmentInstanceInfo( } } - public class PipelineExecContexts { + public static class PipelineExecContexts { + TUniqueId queryId; long beId; + Backend backend; TNetworkAddress brpcAddr; List ctxs = Lists.newArrayList(); boolean twoPhaseExecution = false; @@ -2923,9 +2926,12 @@ public class PipelineExecContexts { boolean hasCancelled = false; boolean cancelInProcess = false; - public PipelineExecContexts(long beId, TNetworkAddress brpcAddr, boolean twoPhaseExecution, + public PipelineExecContexts(TUniqueId queryId, + Backend backend, TNetworkAddress brpcAddr, boolean twoPhaseExecution, int instanceNumber) { - this.beId = beId; + this.queryId = queryId; + this.backend = backend; + this.beId = backend.getId(); this.brpcAddr = brpcAddr; this.twoPhaseExecution = twoPhaseExecution; this.instanceNumber = instanceNumber; @@ -2939,6 +2945,14 @@ public int getInstanceNumber() { return instanceNumber; } + public List getCtxs() { + return ctxs; + } + + public Backend getBackend() { + return backend; + } + /** * The BackendExecState in states are all send to the same BE. * So only the first BackendExecState need to carry some common fields, such as DescriptorTbl, @@ -3014,6 +3028,14 @@ public PExecPlanFragmentResult get(long timeout, TimeUnit unit) { }; } + public void setSerializeFragments(ByteString serializedFragments) { + this.serializedFragments = serializedFragments; + } + + public ByteString getSerializedFragments() { + return serializedFragments; + } + public long serializeFragments() throws TException { TPipelineFragmentParamsList paramsList = new TPipelineFragmentParamsList(); for (PipelineExecContext cts : ctxs) { @@ -3039,7 +3061,7 @@ public String debugInfo() { private synchronized void cancelQuery(Status cancelReason) { if (LOG.isDebugEnabled()) { LOG.debug("cancelRemoteFragments backend: {}, query={}, reason: {}", - idToBackend.get(beId), DebugUtil.printId(queryId), cancelReason.toString()); + backend, DebugUtil.printId(queryId), cancelReason.toString()); } if (this.hasCancelled || this.cancelInProcess) { @@ -3060,7 +3082,7 @@ public void onSuccess(InternalService.PCancelPlanFragmentResult result) { hasCancelled = true; } else { LOG.warn("Failed to cancel query {} backend: {}, reason: {}", - DebugUtil.printId(queryId), idToBackend.get(beId), status.toString()); + DebugUtil.printId(queryId), backend, status.toString()); } } } @@ -3068,7 +3090,7 @@ public void onSuccess(InternalService.PCancelPlanFragmentResult result) { public void onFailure(Throwable t) { cancelInProcess = false; LOG.warn("Failed to cancel query {} backend: {}, reason: {}", - DebugUtil.printId(queryId), idToBackend.get(beId), cancelReason.toString(), t); + DebugUtil.printId(queryId), backend, cancelReason.toString(), t); } }, backendRpcCallbackExecutor); cancelInProcess = true; @@ -3386,15 +3408,11 @@ public List getInvolvedBackends() { return backendAddresses; } - public Map getFragmentExecParamsMap() { - return fragmentExecParamsMap; - } - public List getFragments() { return fragments; } - private void updateProfileIfPresent(Consumer profileAction) { + protected void updateProfileIfPresent(Consumer profileAction) { Optional.ofNullable(context).map(ConnectContext::getExecutor).map(StmtExecutor::getSummaryProfile) .ifPresent(profileAction); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java new file mode 100644 index 00000000000000..dc9678d1145035 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/JobProcessor.java @@ -0,0 +1,27 @@ +// 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.qe; + +import org.apache.doris.common.Status; +import org.apache.doris.qe.runtime.SqlPipelineTask; + +public interface JobProcessor { + void setSqlPipelineTask(SqlPipelineTask sqlPipelineTask); + + void cancel(Status cancelReason); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/LoadContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/LoadContext.java new file mode 100644 index 00000000000000..958aa6d1122d31 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/LoadContext.java @@ -0,0 +1,154 @@ +// 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.qe; + +import org.apache.doris.common.Config; +import org.apache.doris.common.Pair; +import org.apache.doris.load.loadv2.LoadJob; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.task.LoadEtlTask; +import org.apache.doris.thrift.TErrorTabletInfo; +import org.apache.doris.thrift.TTabletCommitInfo; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class LoadContext { + + private volatile String trackingUrl; + private volatile long transactionId; + private volatile String label; + private final List exportFiles = Lists.newCopyOnWriteArrayList(); + private final Map loadCounters = Maps.newLinkedHashMap(); + private final List deltaUrls = Lists.newCopyOnWriteArrayList(); + private final List errorTabletInfos = Lists.newCopyOnWriteArrayList(); + + // in pipelinex, the commit info may be duplicate, so we remove the duplicate ones + // key: backendsId + // values: tabletId + private final Map, TTabletCommitInfo> commitInfoMap = Maps.newLinkedHashMap(); + + public synchronized Map getLoadCounters() { + return ImmutableMap.copyOf(loadCounters); + } + + public synchronized void updateLoadCounters(Map newLoadCounters) { + long numRowsNormal = 0L; + String value = this.loadCounters.get(LoadEtlTask.DPP_NORMAL_ALL); + if (value != null) { + numRowsNormal = Long.parseLong(value); + } + long numRowsAbnormal = 0L; + value = this.loadCounters.get(LoadEtlTask.DPP_ABNORMAL_ALL); + if (value != null) { + numRowsAbnormal = Long.parseLong(value); + } + long numRowsUnselected = 0L; + value = this.loadCounters.get(LoadJob.UNSELECTED_ROWS); + if (value != null) { + numRowsUnselected = Long.parseLong(value); + } + + // new load counters + value = newLoadCounters.get(LoadEtlTask.DPP_NORMAL_ALL); + if (value != null) { + numRowsNormal += Long.parseLong(value); + } + value = newLoadCounters.get(LoadEtlTask.DPP_ABNORMAL_ALL); + if (value != null) { + numRowsAbnormal += Long.parseLong(value); + } + value = newLoadCounters.get(LoadJob.UNSELECTED_ROWS); + if (value != null) { + numRowsUnselected += Long.parseLong(value); + } + + this.loadCounters.put(LoadEtlTask.DPP_NORMAL_ALL, Long.toString(numRowsNormal)); + this.loadCounters.put(LoadEtlTask.DPP_ABNORMAL_ALL, Long.toString(numRowsAbnormal)); + this.loadCounters.put(LoadJob.UNSELECTED_ROWS, Long.toString(numRowsUnselected)); + } + + public List getDeltaUrls() { + return Utils.fastToImmutableList(deltaUrls); + } + + public void updateDeltaUrls(List deltaUrls) { + if (!deltaUrls.isEmpty()) { + this.deltaUrls.addAll(deltaUrls); + } + } + + public synchronized void updateCommitInfos(List commitInfos) { + // distinct commit info in the map + for (TTabletCommitInfo commitInfo : commitInfos) { + this.commitInfoMap.put(Pair.of(commitInfo.backendId, commitInfo.tabletId), commitInfo); + } + } + + public synchronized List getCommitInfos() { + return Utils.fastToImmutableList(commitInfoMap.values()); + } + + public void updateTrackingUrl(String trackingUrl) { + this.trackingUrl = trackingUrl; + } + + public String getTrackingUrl() { + return trackingUrl; + } + + public void updateTransactionId(long transactionId) { + this.transactionId = transactionId; + } + + public long getTransactionId() { + return transactionId; + } + + public String getLabel() { + return label; + } + + public void updateLabel(String label) { + this.label = label; + } + + public void addExportFiles(List files) { + this.exportFiles.addAll(files); + } + + public List getExportFiles() { + return exportFiles; + } + + public synchronized void updateErrorTabletInfos(List errorTabletInfos) { + if (this.errorTabletInfos.size() <= Config.max_error_tablet_of_broker_load) { + this.errorTabletInfos.addAll(errorTabletInfos.stream().limit(Config.max_error_tablet_of_broker_load + - this.errorTabletInfos.size()).collect(Collectors.toList())); + } + } + + public List getErrorTabletInfos() { + return errorTabletInfos; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java deleted file mode 100644 index 4f5af3762c51af..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java +++ /dev/null @@ -1,194 +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.qe; - -import org.apache.doris.analysis.Analyzer; -import org.apache.doris.datasource.FileQueryScanNode; -import org.apache.doris.nereids.NereidsPlanner; -import org.apache.doris.nereids.stats.StatsErrorEstimator; -import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan; -import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping; -import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; -import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.LocalShuffleAssignedJob; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanRanges; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanSource; -import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; -import org.apache.doris.planner.OlapScanNode; -import org.apache.doris.planner.PlanFragment; -import org.apache.doris.planner.Planner; -import org.apache.doris.planner.ScanNode; -import org.apache.doris.thrift.TNetworkAddress; -import org.apache.doris.thrift.TScanRangeParams; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; - -/** NereidsCoordinator */ -public class NereidsCoordinator extends Coordinator { - private NereidsPlanner nereidsPlanner; - private FragmentIdMapping distributedPlans; - - public NereidsCoordinator(ConnectContext context, Analyzer analyzer, - Planner planner, StatsErrorEstimator statsErrorEstimator, NereidsPlanner nereidsPlanner) { - super(context, analyzer, planner, statsErrorEstimator); - this.nereidsPlanner = Objects.requireNonNull(nereidsPlanner, "nereidsPlanner can not be null"); - this.distributedPlans = Objects.requireNonNull( - nereidsPlanner.getDistributedPlans(), "distributedPlans can not be null" - ); - } - - @Override - protected void processFragmentAssignmentAndParams() throws Exception { - // prepare information - prepare(); - - computeFragmentExecParams(); - } - - @Override - protected void computeFragmentHosts() { - // translate distributed plan to params - for (DistributedPlan distributedPlan : distributedPlans.values()) { - UnassignedJob fragmentJob = distributedPlan.getFragmentJob(); - PlanFragment fragment = fragmentJob.getFragment(); - - bucketShuffleJoinController - .isBucketShuffleJoin(fragment.getFragmentId().asInt(), fragment.getPlanRoot()); - - setFileScanParams(distributedPlan); - - FragmentExecParams fragmentExecParams = fragmentExecParamsMap.computeIfAbsent( - fragment.getFragmentId(), id -> new FragmentExecParams(fragment) - ); - List instanceJobs = ((PipelineDistributedPlan) distributedPlan).getInstanceJobs(); - boolean useLocalShuffle = useLocalShuffle(distributedPlan); - if (useLocalShuffle) { - fragmentExecParams.ignoreDataDistribution = true; - fragmentExecParams.parallelTasksNum = 1; - } else { - fragmentExecParams.parallelTasksNum = instanceJobs.size(); - } - - for (AssignedJob instanceJob : instanceJobs) { - DistributedPlanWorker worker = instanceJob.getAssignedWorker(); - TNetworkAddress address = new TNetworkAddress(worker.host(), worker.port()); - FInstanceExecParam instanceExecParam = new FInstanceExecParam( - null, address, 0, fragmentExecParams); - instanceExecParam.instanceId = instanceJob.instanceId(); - fragmentExecParams.instanceExecParams.add(instanceExecParam); - addressToBackendID.put(address, worker.id()); - ScanSource scanSource = instanceJob.getScanSource(); - if (scanSource instanceof BucketScanSource) { - setForBucketScanSource(instanceExecParam, (BucketScanSource) scanSource, useLocalShuffle); - } else { - setForDefaultScanSource(instanceExecParam, (DefaultScanSource) scanSource, useLocalShuffle); - } - } - } - } - - private void setFileScanParams(DistributedPlan distributedPlan) { - for (ScanNode scanNode : distributedPlan.getFragmentJob().getScanNodes()) { - if (scanNode instanceof FileQueryScanNode) { - fileScanRangeParamsMap.put( - scanNode.getId().asInt(), - ((FileQueryScanNode) scanNode).getFileScanRangeParams() - ); - } - } - } - - private boolean useLocalShuffle(DistributedPlan distributedPlan) { - List instanceJobs = ((PipelineDistributedPlan) distributedPlan).getInstanceJobs(); - for (AssignedJob instanceJob : instanceJobs) { - if (instanceJob instanceof LocalShuffleAssignedJob) { - return true; - } - } - return false; - } - - private void setForDefaultScanSource( - FInstanceExecParam instanceExecParam, DefaultScanSource scanSource, boolean isShareScan) { - for (Entry scanNodeIdToReplicaIds : scanSource.scanNodeToScanRanges.entrySet()) { - ScanNode scanNode = scanNodeIdToReplicaIds.getKey(); - ScanRanges scanReplicas = scanNodeIdToReplicaIds.getValue(); - instanceExecParam.perNodeScanRanges.put(scanNode.getId().asInt(), scanReplicas.params); - instanceExecParam.perNodeSharedScans.put(scanNode.getId().asInt(), isShareScan); - } - } - - private void setForBucketScanSource(FInstanceExecParam instanceExecParam, - BucketScanSource bucketScanSource, boolean isShareScan) { - for (Entry> bucketIndexToScanTablets : - bucketScanSource.bucketIndexToScanNodeToTablets.entrySet()) { - Integer bucketIndex = bucketIndexToScanTablets.getKey(); - instanceExecParam.addBucketSeq(bucketIndex); - Map scanNodeToRangeMap = bucketIndexToScanTablets.getValue(); - for (Entry scanNodeToRange : scanNodeToRangeMap.entrySet()) { - ScanNode scanNode = scanNodeToRange.getKey(); - ScanRanges scanRanges = scanNodeToRange.getValue(); - List scanBucketTablets = instanceExecParam.perNodeScanRanges.computeIfAbsent( - scanNode.getId().asInt(), id -> Lists.newArrayList()); - scanBucketTablets.addAll(scanRanges.params); - instanceExecParam.perNodeSharedScans.put(scanNode.getId().asInt(), isShareScan); - - if (scanNode instanceof OlapScanNode) { - OlapScanNode olapScanNode = (OlapScanNode) scanNode; - if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { - int bucketNum = olapScanNode.getBucketNum(); - fragmentIdToSeqToAddressMap.put(olapScanNode.getFragmentId(), new HashMap<>()); - bucketShuffleJoinController.fragmentIdBucketSeqToScanRangeMap - .put(scanNode.getFragmentId(), new BucketSeqToScanRange()); - bucketShuffleJoinController.fragmentIdToBucketNumMap - .put(scanNode.getFragmentId(), bucketNum); - olapScanNode.getFragment().setBucketNum(bucketNum); - } - } else if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { - int bucketNum = 1; - fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); - bucketShuffleJoinController.fragmentIdBucketSeqToScanRangeMap - .put(scanNode.getFragmentId(), new BucketSeqToScanRange()); - bucketShuffleJoinController.fragmentIdToBucketNumMap - .put(scanNode.getFragmentId(), bucketNum); - scanNode.getFragment().setBucketNum(bucketNum); - } - - BucketSeqToScanRange bucketSeqToScanRange = bucketShuffleJoinController - .fragmentIdBucketSeqToScanRangeMap.get(scanNode.getFragmentId()); - - Map> scanNodeIdToReplicas - = bucketSeqToScanRange.computeIfAbsent(bucketIndex, set -> Maps.newLinkedHashMap()); - List tablets = scanNodeIdToReplicas.computeIfAbsent( - scanNode.getId().asInt(), id -> new ArrayList<>()); - tablets.addAll(scanRanges.params); - } - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsSqlCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsSqlCoordinator.java new file mode 100644 index 00000000000000..15c133808c9d5c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsSqlCoordinator.java @@ -0,0 +1,502 @@ +// 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.qe; + +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.analysis.DescriptorTable; +import org.apache.doris.analysis.StorageBackend; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.FsBroker; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Config; +import org.apache.doris.common.Status; +import org.apache.doris.common.UserException; +import org.apache.doris.common.profile.ExecutionProfile; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.stats.StatsErrorEstimator; +import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.BackendWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ResultFileSink; +import org.apache.doris.planner.ResultSink; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.planner.SchemaScanNode; +import org.apache.doris.qe.ConnectContext.ConnectType; +import org.apache.doris.qe.QueryStatisticsItem.FragmentInstanceInfo; +import org.apache.doris.qe.runtime.LoadProcessor; +import org.apache.doris.qe.runtime.MultiFragmentsPipelineTask; +import org.apache.doris.qe.runtime.QueryProcessor; +import org.apache.doris.qe.runtime.SingleFragmentPipelineTask; +import org.apache.doris.qe.runtime.SqlPipelineTask; +import org.apache.doris.qe.runtime.SqlPipelineTaskBuilder; +import org.apache.doris.qe.runtime.ThriftPlansBuilder; +import org.apache.doris.resource.workloadgroup.QueryQueue; +import org.apache.doris.resource.workloadgroup.QueueToken; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TErrorTabletInfo; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TPipelineFragmentParamsList; +import org.apache.doris.thrift.TPipelineWorkloadGroup; +import org.apache.doris.thrift.TQueryOptions; +import org.apache.doris.thrift.TQueryType; +import org.apache.doris.thrift.TReportExecStatusParams; +import org.apache.doris.thrift.TTabletCommitInfo; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +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 java.util.Comparator; +import java.util.List; +import java.util.Map; + +/** NereidsSqlCoordinator */ +public class NereidsSqlCoordinator extends Coordinator { + private static final Logger LOG = LogManager.getLogger(NereidsSqlCoordinator.class); + + protected final SqlCoordinatorContext coordinatorContext; + + protected volatile SqlPipelineTask executionTask; + + public NereidsSqlCoordinator(ConnectContext context, Analyzer analyzer, + NereidsPlanner planner, StatsErrorEstimator statsErrorEstimator) { + super(context, analyzer, planner, statsErrorEstimator); + + this.coordinatorContext = SqlCoordinatorContext.buildForSql(planner, this); + this.coordinatorContext.setJobProcessor(buildJobProcessor(coordinatorContext)); + + Preconditions.checkState(!planner.getFragments().isEmpty() + && coordinatorContext.instanceNum.get() > 0, "Fragment and Instance can not be emptyËš"); + } + + public NereidsSqlCoordinator(Long jobId, TUniqueId queryId, DescriptorTable descTable, + List fragments, List distributedPlans, + List scanNodes, String timezone, boolean loadZeroTolerance, + boolean enableProfile) { + super(jobId, queryId, descTable, fragments, scanNodes, timezone, loadZeroTolerance, enableProfile); + this.coordinatorContext = SqlCoordinatorContext.buildForLoad( + this, jobId, queryId, fragments, distributedPlans, scanNodes, + descTable, timezone, loadZeroTolerance, enableProfile + ); + } + + @Override + public void exec() throws Exception { + enqueue(coordinatorContext.connectContext); + + processTopSink(coordinatorContext, coordinatorContext.topDistributedPlan); + + QeProcessorImpl.INSTANCE.registerInstances(coordinatorContext.queryId, coordinatorContext.instanceNum.get()); + + Map workerToFragments + = ThriftPlansBuilder.plansToThrift(coordinatorContext); + executionTask = SqlPipelineTaskBuilder.build(coordinatorContext, workerToFragments); + executionTask.execute(); + } + + @Override + public boolean isTimeout() { + return System.currentTimeMillis() > coordinatorContext.timeoutDeadline.get(); + } + + @Override + public void cancel(Status cancelReason) { + coordinatorContext.getQueueToken().ifPresent(QueueToken::cancel); + + for (ScanNode scanNode : coordinatorContext.scanNodes) { + scanNode.stop(); + } + + if (cancelReason.ok()) { + throw new RuntimeException("Should use correct cancel reason, but it is " + cancelReason); + } + + TUniqueId queryId = coordinatorContext.queryId; + Status originQueryStatus = coordinatorContext.updateStatusIfOk(cancelReason); + if (!originQueryStatus.ok()) { + if (LOG.isDebugEnabled()) { + // Print an error stack here to know why send cancel again. + LOG.warn("Query {} already in abnormal status {}, but received cancel again," + + "so that send cancel to BE again", + DebugUtil.printId(queryId), originQueryStatus.toString(), + new Exception("cancel failed")); + } + } else { + LOG.warn("Cancel execution of query {}, this is a outside invoke, cancelReason {}", + DebugUtil.printId(queryId), cancelReason); + } + cancelInternal(cancelReason); + } + + public QueryProcessor asQueryProcessor() { + return coordinatorContext.asQueryProcessor(); + } + + public JobProcessor getJobProcessor() { + return coordinatorContext.getJobProcessor(); + } + + public LoadProcessor asLoadProcessor() { + return coordinatorContext.asLoadProcessor(); + } + + @Override + public void setTWorkloadGroups(List tWorkloadGroups) { + coordinatorContext.setWorkloadGroups(tWorkloadGroups); + } + + @Override + public List getTWorkloadGroups() { + return coordinatorContext.getWorkloadGroups(); + } + + @Override + public boolean isQueryCancelled() { + return coordinatorContext.readCloneStatus().isCancelled(); + } + + @Override + public RowBatch getNext() throws Exception { + return coordinatorContext.asQueryProcessor().getNext(); + } + + public boolean isEos() { + return coordinatorContext.asQueryProcessor().isEos(); + } + + @Override + public long getNumReceivedRows() { + return coordinatorContext.asQueryProcessor().getNumReceivedRows(); + } + + @Override + public long getJobId() { + return coordinatorContext.asLoadProcessor().jobId; + } + + /* + * Waiting the coordinator finish executing. + * return false if waiting timeout. + * return true otherwise. + * NOTICE: return true does not mean that coordinator executed success, + * the caller should check queryStatus for result. + * + * We divide the entire waiting process into multiple rounds, + * with a maximum of 30 seconds per round. And after each round of waiting, + * check the status of the BE. If the BE status is abnormal, the wait is ended + * and the result is returned. Otherwise, continue to the next round of waiting. + * This method mainly avoids the problem that the Coordinator waits for a long time + * after some BE can no long return the result due to some exception, such as BE is down. + */ + @Override + public boolean join(int timeoutS) { + return coordinatorContext.asLoadProcessor().join(timeoutS); + } + + @Override + public boolean isDone() { + return coordinatorContext.asLoadProcessor().isDone(); + } + + @Override + public void updateFragmentExecStatus(TReportExecStatusParams params) { + JobProcessor jobProcessor = coordinatorContext.getJobProcessor(); + if (jobProcessor instanceof LoadProcessor) { + coordinatorContext.asLoadProcessor().updateFragmentExecStatus(params); + } + } + + @Override + public TUniqueId getQueryId() { + return coordinatorContext.queryId; + } + + @Override + public TQueryOptions getQueryOptions() { + return coordinatorContext.queryOptions; + } + + @Override + public Status getExecStatus() { + return coordinatorContext.readCloneStatus(); + } + + @Override + public void setQueryType(TQueryType type) { + coordinatorContext.queryOptions.setQueryType(type); + } + + @Override + public void setLoadZeroTolerance(boolean loadZeroTolerance) { + coordinatorContext.queryGlobals.setLoadZeroTolerance(loadZeroTolerance); + } + + @Override + public int getScanRangeNum() { + return coordinatorContext.scanRangeNum.get(); + } + + @Override + public ConnectContext getConnectContext() { + return coordinatorContext.connectContext; + } + + @Override + public QueueToken getQueueToken() { + return coordinatorContext.getQueueToken().orElse(null); + } + + @Override + public Map getLoadCounters() { + return coordinatorContext.asLoadProcessor().loadContext.getLoadCounters(); + } + + @Override + public List getDeltaUrls() { + return coordinatorContext.asLoadProcessor().loadContext.getDeltaUrls(); + } + + @Override + public List getCommitInfos() { + return coordinatorContext.asLoadProcessor().loadContext.getCommitInfos(); + } + + @Override + public List getExportFiles() { + return coordinatorContext.asLoadProcessor().loadContext.getExportFiles(); + } + + @Override + public long getTxnId() { + return coordinatorContext.asLoadProcessor().loadContext.getTransactionId(); + } + + @Override + public void setTxnId(long txnId) { + coordinatorContext.asLoadProcessor().loadContext.updateTransactionId(txnId); + } + + @Override + public String getLabel() { + return coordinatorContext.asLoadProcessor().loadContext.getLabel(); + } + + @Override + public String getTrackingUrl() { + return coordinatorContext.asLoadProcessor().loadContext.getTrackingUrl(); + } + + @Override + public List getErrorTabletInfos() { + return coordinatorContext.asLoadProcessor().loadContext.getErrorTabletInfos(); + } + + @Override + public List getInvolvedBackends() { + return Utils.fastToImmutableList(coordinatorContext.backends.get().keySet()); + } + + @Override + public List getFragmentInstanceInfos() { + List infos = Lists.newArrayList(); + if (executionTask != null) { + for (MultiFragmentsPipelineTask multiFragmentsPipelineTask : executionTask.getChildrenTasks().values()) { + for (SingleFragmentPipelineTask fragmentTask : multiFragmentsPipelineTask.getChildrenTasks().values()) { + infos.addAll(fragmentTask.buildFragmentInstanceInfo()); + } + } + } + infos.sort(Comparator.comparing(FragmentInstanceInfo::getFragmentId)); + return infos; + } + + @Override + public List getFragments() { + return coordinatorContext.fragments; + } + + @Override + public ExecutionProfile getExecutionProfile() { + return coordinatorContext.executionProfile; + } + + @Override + public void setMemTableOnSinkNode(boolean enableMemTableOnSinkNode) { + coordinatorContext.queryOptions.setEnableMemtableOnSinkNode(enableMemTableOnSinkNode); + } + + @Override + public void setBatchSize(int batchSize) { + coordinatorContext.queryOptions.setBatchSize(batchSize); + } + + @Override + public void setTimeout(int timeout) { + coordinatorContext.queryOptions.setQueryTimeout(timeout); + coordinatorContext.queryOptions.setExecutionTimeout(timeout); + if (coordinatorContext.queryOptions.getExecutionTimeout() < 1) { + LOG.warn("try set timeout less than 1: {}", coordinatorContext.queryOptions.getExecutionTimeout()); + } + } + + @Override + public void setLoadMemLimit(long loadMemLimit) { + coordinatorContext.queryOptions.setLoadMemLimit(loadMemLimit); + } + + @Override + public void setExecMemoryLimit(long execMemoryLimit) { + coordinatorContext.queryOptions.setMemLimit(execMemoryLimit); + } + + // this method is used to provide profile metrics: `Instances Num Per BE` + @Override + public Map getBeToInstancesNum() { + Map result = Maps.newLinkedHashMap(); + if (executionTask != null) { + for (MultiFragmentsPipelineTask beTasks : executionTask.getChildrenTasks().values()) { + TNetworkAddress brpcAddress = beTasks.getBackend().getBrpcAddress(); + String brpcAddrString = brpcAddress.hostname.concat(":").concat("" + brpcAddress.port); + result.put(brpcAddrString, beTasks.getChildrenTasks().size()); + } + } + return result; + } + + @Override + public void close() { + // NOTE: all close method should be no exception + if (coordinatorContext.getQueryQueue().isPresent() && coordinatorContext.getQueueToken().isPresent()) { + try { + coordinatorContext.getQueryQueue().get().releaseAndNotify(coordinatorContext.getQueueToken().get()); + } catch (Throwable t) { + LOG.error("error happens when coordinator close ", t); + } + } + + try { + for (ScanNode scanNode : coordinatorContext.scanNodes) { + scanNode.stop(); + } + } catch (Throwable t) { + LOG.error("error happens when scannode stop ", t); + } + } + + protected void cancelInternal(Status cancelReason) { + coordinatorContext.withLock(() -> coordinatorContext.getJobProcessor().cancel(cancelReason)); + } + + protected void processTopSink( + SqlCoordinatorContext coordinatorContext, PipelineDistributedPlan topPlan) throws AnalysisException { + setForArrowFlight(coordinatorContext, topPlan); + setForBroker(coordinatorContext, topPlan); + } + + private void setForArrowFlight(SqlCoordinatorContext coordinatorContext, PipelineDistributedPlan topPlan) { + ConnectContext connectContext = coordinatorContext.connectContext; + DataSink dataSink = coordinatorContext.dataSink; + if (dataSink instanceof ResultSink || dataSink instanceof ResultFileSink) { + if (connectContext != null && !connectContext.isReturnResultFromLocal()) { + Preconditions.checkState(connectContext.getConnectType().equals(ConnectType.ARROW_FLIGHT_SQL)); + + AssignedJob firstInstance = topPlan.getInstanceJobs().get(0); + BackendWorker worker = (BackendWorker) firstInstance.getAssignedWorker(); + Backend backend = worker.getBackend(); + + connectContext.setFinstId(firstInstance.instanceId()); + if (backend.getArrowFlightSqlPort() < 0) { + throw new IllegalStateException("be arrow_flight_sql_port cannot be empty."); + } + connectContext.setResultFlightServerAddr(backend.getArrowFlightAddress()); + connectContext.setResultInternalServiceAddr(backend.getBrpcAddress()); + connectContext.setResultOutputExprs(topPlan.getFragmentJob().getFragment().getOutputExprs()); + } + } + } + + private void setForBroker( + SqlCoordinatorContext coordinatorContext, PipelineDistributedPlan topPlan) throws AnalysisException { + DataSink dataSink = coordinatorContext.dataSink; + if (dataSink instanceof ResultFileSink + && ((ResultFileSink) dataSink).getStorageType() == StorageBackend.StorageType.BROKER) { + // set the broker address for OUTFILE sink + ResultFileSink topResultFileSink = (ResultFileSink) dataSink; + DistributedPlanWorker worker = topPlan.getInstanceJobs().get(0).getAssignedWorker(); + FsBroker broker = Env.getCurrentEnv().getBrokerMgr() + .getBroker(topResultFileSink.getBrokerName(), worker.host()); + topResultFileSink.setBrokerAddr(broker.host, broker.port); + } + } + + private void enqueue(ConnectContext context) throws UserException { + // LoadTask does not have context, not controlled by queue now + if (context != null) { + if (Config.enable_workload_group) { + coordinatorContext.setWorkloadGroups(context.getEnv().getWorkloadGroupMgr().getWorkloadGroup(context)); + if (shouldQueue(context)) { + QueryQueue queryQueue = context.getEnv().getWorkloadGroupMgr().getWorkloadGroupQueryQueue(context); + if (queryQueue == null) { + // This logic is actually useless, because when could not find query queue, it will + // throw exception during workload group manager. + throw new UserException("could not find query queue"); + } + QueueToken queueToken = queryQueue.getToken(); + int queryTimeout = coordinatorContext.queryOptions.getExecutionTimeout() * 1000; + queueToken.get(DebugUtil.printId(coordinatorContext.queryId), queryTimeout); + coordinatorContext.setQueueInfo(queryQueue, queueToken); + } + } else { + context.setWorkloadGroupName(""); + } + } + } + + private boolean shouldQueue(ConnectContext context) { + boolean ret = Config.enable_query_queue && !context.getSessionVariable() + .getBypassWorkloadGroup() && !isQueryCancelled(); + if (!ret) { + return false; + } + // a query with ScanNode need not queue only when all its scan node is SchemaScanNode + for (ScanNode scanNode : coordinatorContext.scanNodes) { + if (!(scanNode instanceof SchemaScanNode)) { + return true; + } + } + return false; + } + + private JobProcessor buildJobProcessor(SqlCoordinatorContext coordinatorContext) { + DataSink dataSink = coordinatorContext.dataSink; + if ((dataSink instanceof ResultSink || dataSink instanceof ResultFileSink)) { + return QueryProcessor.build(coordinatorContext); + } else { + // insert statement has jobId == -1 + return new LoadProcessor(coordinatorContext, -1L); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java index ffbc9744ca4e78..f1304826e39d2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -53,7 +53,7 @@ public class ResultReceiver { private Types.PUniqueId finstId; private Long backendId; private Thread currentThread; - private Future fetchDataAsyncFuture = null; + private volatile Future fetchDataAsyncFuture = null; private Boolean enableParallelResultSink = false; int maxMsgSizeOfResultReceiver; 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 cc1f29b76c2b49..caba12515c0b19 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 @@ -34,6 +34,8 @@ import org.apache.doris.nereids.metrics.EventSwitchParser; import org.apache.doris.nereids.parser.Dialect; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.planner.GroupCommitBlockSink; import org.apache.doris.qe.VariableMgr.VarAttr; @@ -1367,7 +1369,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { + "right side to do bucket shuffle join" } ) - private boolean enableNereidsDistributePlanner = false; + private boolean enableNereidsDistributePlanner = true; @VariableMgr.VarAttr(name = REWRITE_OR_TO_IN_PREDICATE_THRESHOLD, fuzzy = true) private int rewriteOrToInPredicateThreshold = 2; @@ -3371,11 +3373,11 @@ public static boolean canUseNereidsDistributePlanner() { } ConnectContext connectContext = ConnectContext.get(); if (connectContext == null) { - return false; + return true; } StatementContext statementContext = connectContext.getStatementContext(); if (statementContext == null) { - return false; + return true; } StatementBase parsedStatement = statementContext.getParsedStatement(); if (!(parsedStatement instanceof LogicalPlanAdapter)) { @@ -3384,7 +3386,9 @@ public static boolean canUseNereidsDistributePlanner() { LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStatement).getLogicalPlan(); SessionVariable sessionVariable = connectContext.getSessionVariable(); // TODO: support other sink - if (logicalPlan instanceof UnboundResultSink && sessionVariable.enableNereidsDistributePlanner) { + if ((logicalPlan instanceof UnboundResultSink + || logicalPlan instanceof LogicalFileSink + || logicalPlan instanceof InsertIntoTableCommand) && sessionVariable.enableNereidsDistributePlanner) { return true; } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SqlCoordinatorContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SqlCoordinatorContext.java new file mode 100644 index 00000000000000..c0d5c625e02fb0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SqlCoordinatorContext.java @@ -0,0 +1,426 @@ +// 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.qe; + +import org.apache.doris.analysis.DescriptorTable; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.Status; +import org.apache.doris.common.profile.ExecutionProfile; +import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.mysql.MysqlCommand; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.physical.TopnFilter; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.RuntimeFilter; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.runtime.LoadProcessor; +import org.apache.doris.qe.runtime.QueryProcessor; +import org.apache.doris.resource.workloadgroup.QueryQueue; +import org.apache.doris.resource.workloadgroup.QueueToken; +import org.apache.doris.service.ExecuteEnv; +import org.apache.doris.thrift.TDescriptorTable; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TPipelineWorkloadGroup; +import org.apache.doris.thrift.TQueryGlobals; +import org.apache.doris.thrift.TQueryOptions; +import org.apache.doris.thrift.TResourceLimit; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.time.LocalDateTime; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class SqlCoordinatorContext { + private static final Logger LOG = LogManager.getLogger(SqlCoordinatorContext.class); + + // these are some constant parameters + public final NereidsSqlCoordinator coordinator; + public final List fragments; + public final boolean isBlockQuery; + public final DataSink dataSink; + public final ExecutionProfile executionProfile; + public final ConnectContext connectContext; + public final PipelineDistributedPlan topDistributedPlan; + public final List distributedPlans; + public final TUniqueId queryId; + public final TQueryGlobals queryGlobals; + public final TQueryOptions queryOptions; + public final TDescriptorTable descriptorTable; + public final TNetworkAddress coordinatorAddress = new TNetworkAddress(Coordinator.localIP, Config.rpc_port); + // public final TNetworkAddress directConnectFrontendAddress; + public final List runtimeFilters; + public final List topnFilters; + public final List scanNodes; + public final Supplier timeoutDeadline = Suppliers.memoize(this::computeTimeoutDeadline); + public final Supplier instanceNum = Suppliers.memoize(this::computeInstanceNum); + public final Supplier> instanceIds = Suppliers.memoize(this::getInstanceIds); + public final Supplier> backends = Suppliers.memoize(this::getBackends); + public final Supplier scanRangeNum = Suppliers.memoize(this::getScanRangeNum); + public final Supplier directConnectFrontendAddress + = Suppliers.memoize(this::computeDirectConnectCoordinator); + + // these are some mutable states + private volatile Status status = new Status(); + private volatile Optional queryQueue = Optional.empty(); + private volatile Optional queueToken = Optional.empty(); + private volatile List workloadGroups = ImmutableList.of(); + + // query or load processor + private volatile JobProcessor jobProcessor; + + // for sql execution + private SqlCoordinatorContext( + NereidsSqlCoordinator coordinator, + ConnectContext connectContext, + boolean isBlockQuery, + List distributedPlans, + List fragments, + List runtimeFilters, + List topnFilters, + List scanNodes, + ExecutionProfile executionProfile, + TQueryGlobals queryGlobals, + TQueryOptions queryOptions, + TDescriptorTable descriptorTable) { + this.connectContext = connectContext; + this.isBlockQuery = isBlockQuery; + this.fragments = fragments; + this.distributedPlans = distributedPlans; + this.topDistributedPlan = distributedPlans.get(distributedPlans.size() - 1); + this.dataSink = topDistributedPlan.getFragmentJob().getFragment().getSink(); + this.runtimeFilters = runtimeFilters == null ? Lists.newArrayList() : runtimeFilters; + this.topnFilters = topnFilters == null ? Lists.newArrayList() : topnFilters; + this.scanNodes = scanNodes; + this.queryId = connectContext.queryId(); + this.executionProfile = executionProfile; + this.queryGlobals = queryGlobals; + this.queryOptions = queryOptions; + this.descriptorTable = descriptorTable; + + this.coordinator = Objects.requireNonNull(coordinator, "coordinator can not be null"); + } + + // for broker load + private SqlCoordinatorContext( + NereidsSqlCoordinator coordinator, + long jobId, + List fragments, + List distributedPlans, + List scanNodes, + TUniqueId queryId, + TQueryOptions queryOptions, + TQueryGlobals queryGlobals, + TDescriptorTable descriptorTable, + ExecutionProfile executionProfile) { + this.coordinator = coordinator; + this.isBlockQuery = true; + this.fragments = fragments; + this.distributedPlans = distributedPlans; + this.topDistributedPlan = distributedPlans.get(distributedPlans.size() - 1); + this.dataSink = topDistributedPlan.getFragmentJob().getFragment().getSink(); + this.scanNodes = scanNodes; + this.queryId = queryId; + this.queryOptions = queryOptions; + this.queryGlobals = queryGlobals; + this.descriptorTable = descriptorTable; + this.executionProfile = executionProfile; + + this.connectContext = ConnectContext.get(); + this.runtimeFilters = ImmutableList.of(); + this.topnFilters = ImmutableList.of(); + this.jobProcessor = new LoadProcessor(this, jobId); + } + + public void setQueueInfo(QueryQueue queryQueue, QueueToken queueToken) { + this.queryQueue = Optional.ofNullable(queryQueue); + this.queueToken = Optional.ofNullable(queueToken); + } + + public Optional getQueueToken() { + return this.queueToken; + } + + public Optional getQueryQueue() { + return queryQueue; + } + + public void setWorkloadGroups(List workloadGroups) { + this.workloadGroups = workloadGroups; + } + + public List getWorkloadGroups() { + return workloadGroups; + } + + public void updateProfileIfPresent(Consumer profileAction) { + Optional.ofNullable(connectContext) + .map(ConnectContext::getExecutor) + .map(StmtExecutor::getSummaryProfile) + .ifPresent(profileAction); + } + + public boolean twoPhaseExecution() { + // If #fragments >=2, use twoPhaseExecution with exec_plan_fragments_prepare and exec_plan_fragments_start, + // else use exec_plan_fragments directly. + // we choose #fragments > 1 because in some cases + // we need ensure that A fragment is already prepared to receive data before B fragment sends data. + // For example: select * from numbers("number"="10") will generate ExchangeNode and + // TableValuedFunctionScanNode, we should ensure TableValuedFunctionScanNode does not + // send data until ExchangeNode is ready to receive. + return distributedPlans.size() > 1; + } + + public boolean isEos() { + return jobProcessor instanceof QueryProcessor && coordinator.isEos(); + } + + public void cancelSchedule(Status cancelReason) { + coordinator.cancelInternal(cancelReason); + } + + public synchronized void withLock(Runnable callback) { + callback.run(); + } + + public synchronized T withLock(Callable callback) throws Exception { + return callback.call(); + } + + public synchronized Status readCloneStatus() { + return new Status(status.getErrorCode(), status.getErrorMsg()); + } + + public synchronized Status updateStatusIfOk(Status newStatus) { + // If query is done, we will ignore their cancelled updates, and let the remote fragments to clean up async. + Status originStatus = readCloneStatus(); + if (coordinator.getJobProcessor() instanceof QueryProcessor && coordinator.isEos() + && newStatus.isCancelled()) { + return originStatus; + } + // nothing to update + if (newStatus.ok()) { + return originStatus; + } + + // don't override an error status; also, cancellation has already started + if (!this.status.ok()) { + return originStatus; + } + + status = new Status(newStatus.getErrorCode(), newStatus.getErrorMsg()); + coordinator.cancelInternal(readCloneStatus()); + return originStatus; + } + + public void setJobProcessor(JobProcessor jobProcessor) { + this.jobProcessor = jobProcessor; + } + + public JobProcessor getJobProcessor() { + return jobProcessor; + } + + public LoadProcessor asLoadProcessor() { + return (LoadProcessor) jobProcessor; + } + + public QueryProcessor asQueryProcessor() { + return (QueryProcessor) jobProcessor; + } + + public static SqlCoordinatorContext buildForSql(NereidsPlanner planner, NereidsSqlCoordinator coordinator) { + ConnectContext connectContext = planner.getCascadesContext().getConnectContext(); + TQueryOptions queryOptions = initQueryOptions(connectContext); + TQueryGlobals queryGlobals = initQueryGlobals(connectContext); + TDescriptorTable descriptorTable = planner.getDescTable().toThrift(); + + ExecutionProfile executionProfile = new ExecutionProfile( + connectContext.queryId, + planner.getFragments() + .stream() + .map(fragment -> fragment.getFragmentId().asInt()) + .collect(Collectors.toList()) + ); + return new SqlCoordinatorContext( + coordinator, connectContext, planner.isBlockQuery(), + planner.getDistributedPlans().valueList(), + planner.getFragments(), planner.getRuntimeFilters(), planner.getTopnFilters(), + planner.getScanNodes(), executionProfile, queryGlobals, queryOptions, descriptorTable + ); + } + + public static SqlCoordinatorContext buildForLoad( + NereidsSqlCoordinator coordinator, + long jobId, TUniqueId queryId, + List fragments, + List distributedPlans, + List scanNodes, + DescriptorTable descTable, + String timezone, boolean loadZeroTolerance, + boolean enableProfile) { + TQueryOptions queryOptions = new TQueryOptions(); + queryOptions.setEnableProfile(enableProfile); + queryOptions.setBeExecVersion(Config.be_exec_version); + + TQueryGlobals queryGlobals = new TQueryGlobals(); + queryGlobals.setNowString(TimeUtils.getDatetimeFormatWithTimeZone().format(LocalDateTime.now())); + queryGlobals.setTimestampMs(System.currentTimeMillis()); + queryGlobals.setTimeZone(timezone); + queryGlobals.setLoadZeroTolerance(loadZeroTolerance); + + ExecutionProfile executionProfile = new ExecutionProfile( + queryId, + fragments.stream() + .map(fragment -> fragment.getFragmentId().asInt()) + .collect(Collectors.toList()) + ); + + return new SqlCoordinatorContext(coordinator, jobId, fragments, distributedPlans, + scanNodes, queryId, queryOptions, queryGlobals, descTable.toThrift(), + executionProfile); + } + + private static TQueryOptions initQueryOptions(ConnectContext context) { + TQueryOptions queryOptions = context.getSessionVariable().toThrift(); + queryOptions.setBeExecVersion(Config.be_exec_version); + queryOptions.setQueryTimeout(context.getExecTimeout()); + queryOptions.setExecutionTimeout(context.getExecTimeout()); + if (queryOptions.getExecutionTimeout() < 1) { + LOG.info("try set timeout less than 1", new RuntimeException("")); + } + queryOptions.setFeProcessUuid(ExecuteEnv.getInstance().getProcessUUID()); + queryOptions.setMysqlRowBinaryFormat(context.getCommand() == MysqlCommand.COM_STMT_EXECUTE); + + setOptionsFromUserProperty(context, queryOptions); + return queryOptions; + } + + private static TQueryGlobals initQueryGlobals(ConnectContext context) { + TQueryGlobals queryGlobals = new TQueryGlobals(); + queryGlobals.setNowString(TimeUtils.getDatetimeFormatWithTimeZone().format(LocalDateTime.now())); + queryGlobals.setTimestampMs(System.currentTimeMillis()); + queryGlobals.setNanoSeconds(LocalDateTime.now().getNano()); + queryGlobals.setLoadZeroTolerance(false); + if (context.getSessionVariable().getTimeZone().equals("CST")) { + queryGlobals.setTimeZone(TimeUtils.DEFAULT_TIME_ZONE); + } else { + queryGlobals.setTimeZone(context.getSessionVariable().getTimeZone()); + } + return queryGlobals; + } + + private static void setOptionsFromUserProperty(ConnectContext connectContext, TQueryOptions queryOptions) { + String qualifiedUser = connectContext.getQualifiedUser(); + // set cpu resource limit + int cpuLimit = Env.getCurrentEnv().getAuth().getCpuResourceLimit(qualifiedUser); + if (cpuLimit > 0) { + // overwrite the cpu resource limit from session variable; + TResourceLimit resourceLimit = new TResourceLimit(); + resourceLimit.setCpuLimit(cpuLimit); + queryOptions.setResourceLimit(resourceLimit); + } + // set exec mem limit + long maxExecMemByte = connectContext.getSessionVariable().getMaxExecMemByte(); + long memLimit = maxExecMemByte > 0 ? maxExecMemByte : + Env.getCurrentEnv().getAuth().getExecMemLimit(qualifiedUser); + if (memLimit > 0) { + // overwrite the exec_mem_limit from session variable; + queryOptions.setMemLimit(memLimit); + queryOptions.setMaxReservation(memLimit); + queryOptions.setInitialReservationTotalClaims(memLimit); + queryOptions.setBufferPoolLimit(memLimit); + } + } + + private Set getInstanceIds() { + Set instanceIds = Sets.newLinkedHashSet(); + for (DistributedPlan distributedPlan : distributedPlans) { + PipelineDistributedPlan pipelinePlan = (PipelineDistributedPlan) distributedPlan; + List instanceJobs = pipelinePlan.getInstanceJobs(); + for (AssignedJob instanceJob : instanceJobs) { + instanceIds.add(instanceJob.instanceId()); + } + } + return instanceIds; + } + + private Integer computeInstanceNum() { + return distributedPlans + .stream() + .map(plan -> plan.getInstanceJobs().size()) + .reduce(Integer::sum) + .get(); + } + + private long computeTimeoutDeadline() { + return System.currentTimeMillis() + queryOptions.getExecutionTimeout() * 1000L; + } + + private Map getBackends() { + Map backends = Maps.newLinkedHashMap(); + for (DistributedPlan distributedPlan : distributedPlans) { + PipelineDistributedPlan pipelinePlan = (PipelineDistributedPlan) distributedPlan; + List instanceJobs = pipelinePlan.getInstanceJobs(); + for (AssignedJob instanceJob : instanceJobs) { + DistributedPlanWorker worker = instanceJob.getAssignedWorker(); + backends.put(new TNetworkAddress(worker.address(), worker.port()), worker.id()); + } + } + return backends; + } + + private TNetworkAddress computeDirectConnectCoordinator() { + if (connectContext != null && connectContext.isProxy() + && !StringUtils.isEmpty(connectContext.getCurrentConnectedFEIp())) { + return new TNetworkAddress(ConnectContext.get().getCurrentConnectedFEIp(), Config.rpc_port); + } else { + return coordinatorAddress; + } + } + + private int getScanRangeNum() { + int scanRangeNum = 0; + for (ScanNode scanNode : scanNodes) { + scanRangeNum += scanNode.getScanRangeNum(); + } + return scanRangeNum; + } +} 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 234d5f0e610780..950fc4fad0a4c2 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 @@ -391,7 +391,7 @@ private Map getSummaryInfo(boolean isFinished) { String taskState = "RUNNING"; if (isFinished) { if (coord != null) { - taskState = coord.queryStatus.getErrorCode().name(); + taskState = coord.getExecStatus().getErrorCode().name(); } else { taskState = context.getState().toString(); } @@ -1907,9 +1907,8 @@ public void executeAndSendResult(boolean isOutfileQuery, boolean isSendFields, coordBase = new PointQueryExecutor(shortCircuitQueryContext, context.getSessionVariable().getMaxMsgSizeOfResultReceiver()); } else if (planner instanceof NereidsPlanner && ((NereidsPlanner) planner).getDistributedPlans() != null) { - coord = new NereidsCoordinator(context, analyzer, - planner, context.getStatsErrorEstimator(), - (NereidsPlanner) planner); + coord = new NereidsSqlCoordinator(context, analyzer, + (NereidsPlanner) planner, context.getStatsErrorEstimator()); profile.addExecutionProfile(coord.getExecutionProfile()); QeProcessorImpl.INSTANCE.registerQuery(context.queryId(), new QueryInfo(context, originStmt.originStmt, coord)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/protocol/TFastSerializer.java b/fe/fe-core/src/main/java/org/apache/doris/qe/protocol/TFastSerializer.java new file mode 100644 index 00000000000000..7a317d23472ab7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/protocol/TFastSerializer.java @@ -0,0 +1,94 @@ +// 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.qe.protocol; + +import com.google.protobuf.ByteString; +import com.google.protobuf.UnsafeByteOperations; +import org.apache.thrift.TBase; +import org.apache.thrift.TConfiguration; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.transport.TIOStreamTransport; +import org.apache.thrift.transport.TTransportException; + +import java.io.ByteArrayOutputStream; + +/** + * Copy from TSerializer and support change the initial byte array capacity + */ +public class TFastSerializer { + + /** + * This is the byte array that data is actually serialized into + */ + // private final NoResizeByteOutputStream baos; + private final ByteArrayOutputStream baos; + + /** + * This transport wraps that byte array + */ + private final TIOStreamTransport transport; + + /** + * Internal protocol used for serializing objects. + */ + private TProtocol protocol; + + /** + * Create a new TSerializer. It will use the TProtocol specified by the + * factory that is passed in. + * + * @param initCapacity init capacity to the write buffer + * @param protocolFactory Factory to create a protocol + * @throws TTransportException if there an error initializing the underlying transport. + */ + public TFastSerializer(int initCapacity, TProtocolFactory protocolFactory) throws TTransportException { + baos = new ByteArrayOutputStream(initCapacity); + transport = new TIOStreamTransport(new TConfiguration(), baos); + protocol = protocolFactory.getProtocol(transport); + } + + /** + * Serialize the Thrift object into a byte array. The process is simple, + * just clear the byte array output, write the object into it, and grab the + * raw bytes. + * + * @param base The object to serialize + * @return Serialized object in byte[] format + * @throws TException if an error is encountered during serialization. + */ + public ByteString serialize(TBase base) throws TException { + baos.reset(); + base.write(protocol); + return UnsafeByteOperations.unsafeWrap(baos.toByteArray()); + } + + /** + * Serialize the Thrift object into a Java string, using the default JVM + * charset encoding. + * + * @param base The object to serialize + * @return Serialized object as a String + * @throws TException if an error is encountered during serialization. + */ + public String toString(TBase base) throws TException { + return new String(serialize(base).toByteArray()); + } +} + diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/AbstractRuntimeTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/AbstractRuntimeTask.java new file mode 100644 index 00000000000000..1059440792e529 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/AbstractRuntimeTask.java @@ -0,0 +1,43 @@ +// 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.qe.runtime; + +import java.util.Map; +import java.util.Objects; + +public abstract class AbstractRuntimeTask> { + protected final ChildrenRuntimeTasks childrenTasks; + + public AbstractRuntimeTask(ChildrenRuntimeTasks childrenTasks) { + this.childrenTasks = Objects.requireNonNull(childrenTasks, "childrenTasks can not be null"); + } + + public void execute() throws Throwable { + for (Child childrenTask : childrenTasks.allTasks()) { + childrenTask.execute(); + } + } + + public Map getChildrenTasks() { + return childrenTasks.allTaskMap(); + } + + protected Child childTask(ChildId childId) { + return childrenTasks.get(childId); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/BackendFragmentId.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/BackendFragmentId.java new file mode 100644 index 00000000000000..26166c54c83c60 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/BackendFragmentId.java @@ -0,0 +1,28 @@ +// 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.qe.runtime; + +import lombok.AllArgsConstructor; +import lombok.Data; + +@Data +@AllArgsConstructor +public class BackendFragmentId { + public final long backendId; + public final int fragmentId; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ChildrenRuntimeTasks.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ChildrenRuntimeTasks.java new file mode 100644 index 00000000000000..b6ac80d1066a0b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ChildrenRuntimeTasks.java @@ -0,0 +1,48 @@ +// 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.qe.runtime; + +import org.apache.doris.nereids.util.Utils; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +public class ChildrenRuntimeTasks> { + // LinkedHashMap: make sure the key set order is same as the input map + // so that we can initial the runtime filter merge backend first + private final Map childrenTasks = Maps.newLinkedHashMap(); + + public ChildrenRuntimeTasks(Map childrenTasks) { + this.childrenTasks.putAll(childrenTasks); + } + + public C get(Id id) { + return childrenTasks.get(id); + } + + public List allTasks() { + return Utils.fastToImmutableList(childrenTasks.values()); + } + + public Map allTaskMap() { + return ImmutableMap.copyOf(childrenTasks); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LeafRuntimeTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LeafRuntimeTask.java new file mode 100644 index 00000000000000..7a9e523b05eeaa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LeafRuntimeTask.java @@ -0,0 +1,26 @@ +// 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.qe.runtime; + +import com.google.common.collect.ImmutableMap; + +public class LeafRuntimeTask extends AbstractRuntimeTask { + public LeafRuntimeTask() { + super(new ChildrenRuntimeTasks<>(ImmutableMap.of())); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java new file mode 100644 index 00000000000000..17151d28ee5a2d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/LoadProcessor.java @@ -0,0 +1,293 @@ +// 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.qe.runtime; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.Status; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.datasource.hive.HMSTransaction; +import org.apache.doris.datasource.iceberg.IcebergTransaction; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.JobProcessor; +import org.apache.doris.qe.LoadContext; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.thrift.TFragmentInstanceReport; +import org.apache.doris.thrift.TReportExecStatusParams; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +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.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +public class LoadProcessor implements JobProcessor { + private static final Logger LOG = LogManager.getLogger(LoadProcessor.class); + + public final SqlCoordinatorContext coordinatorContext; + public final LoadContext loadContext; + public final long jobId; + + // this latch is used to wait finish for load, for example, insert into statement + // MarkedCountDownLatch: + // key: fragmentId, value: backendId + private volatile Optional executionTask; + private volatile Optional> latch; + private volatile Optional> backendFragmentTasks; + private volatile List topFragmentTasks; + + public LoadProcessor(SqlCoordinatorContext coordinatorContext, long jobId) { + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.loadContext = new LoadContext(); + this.executionTask = Optional.empty(); + this.latch = Optional.empty(); + this.backendFragmentTasks = Optional.empty(); + + // only we set is report success, then the backend would report the fragment status, + // then we can not the fragment is finished, and we can return in the NereidsCoordinator::join + coordinatorContext.queryOptions.setIsReportSuccess(true); + // the insert into statement isn't a job + this.jobId = jobId; + TUniqueId queryId = coordinatorContext.queryId; + Env.getCurrentEnv().getLoadManager().initJobProgress( + jobId, queryId, coordinatorContext.instanceIds.get(), + Utils.fastToImmutableList(coordinatorContext.backends.get().values()) + ); + Env.getCurrentEnv().getProgressManager().addTotalScanNums( + String.valueOf(jobId), coordinatorContext.scanRangeNum.get() + ); + + topFragmentTasks = Lists.newArrayList(); + + LOG.info("dispatch load job: {} to {}", DebugUtil.printId(queryId), coordinatorContext.backends.get().keySet()); + } + + @Override + public void setSqlPipelineTask(SqlPipelineTask sqlPipelineTask) { + Preconditions.checkArgument(sqlPipelineTask != null, "sqlPipelineTask can not be null"); + + this.executionTask = Optional.of(sqlPipelineTask); + Map backendFragmentTasks + = buildBackendFragmentTasks(sqlPipelineTask); + this.backendFragmentTasks = Optional.of(backendFragmentTasks); + + MarkedCountDownLatch latch = new MarkedCountDownLatch<>(backendFragmentTasks.size()); + for (BackendFragmentId backendFragmentId : backendFragmentTasks.keySet()) { + latch.addMark(backendFragmentId.fragmentId, backendFragmentId.backendId); + } + this.latch = Optional.of(latch); + + int topFragmentId = coordinatorContext.topDistributedPlan + .getFragmentJob() + .getFragment() + .getFragmentId() + .asInt(); + List topFragmentTasks = Lists.newArrayList(); + for (MultiFragmentsPipelineTask multiFragmentPipelineTask : sqlPipelineTask.childrenTasks.allTasks()) { + for (SingleFragmentPipelineTask fragmentTask : multiFragmentPipelineTask.childrenTasks.allTasks()) { + if (fragmentTask.getFragmentId() == topFragmentId) { + topFragmentTasks.add(fragmentTask); + } + } + } + this.topFragmentTasks = topFragmentTasks; + } + + @Override + public void cancel(Status cancelReason) { + if (executionTask.isPresent()) { + for (MultiFragmentsPipelineTask fragmentsTask : executionTask.get().getChildrenTasks().values()) { + fragmentsTask.cancelExecute(cancelReason); + } + latch.get().countDownToZero(new Status()); + } + } + + public boolean isDone() { + return latch.map(l -> l.getCount() == 0).orElse(false); + } + + public boolean join(int timeoutS) { + SqlPipelineTask sqlPipelineTask = this.executionTask.orElse(null); + if (sqlPipelineTask == null) { + return true; + } + + long fixedMaxWaitTime = 30; + + long leftTimeoutS = timeoutS; + while (leftTimeoutS > 0) { + long waitTime = Math.min(leftTimeoutS, fixedMaxWaitTime); + boolean awaitRes = false; + try { + awaitRes = await(waitTime, TimeUnit.SECONDS); + } catch (InterruptedException e) { + // Do nothing + } + if (awaitRes) { + return true; + } + + if (!checkHealthy()) { + return true; + } + + leftTimeoutS -= waitTime; + } + return false; + } + + public boolean await(long timeout, TimeUnit unit) throws InterruptedException { + if (!latch.isPresent()) { + return false; + } + return latch.get().await(timeout, unit); + } + + public void updateFragmentExecStatus(TReportExecStatusParams params) { + SingleFragmentPipelineTask fragmentTask = backendFragmentTasks.get().get( + new BackendFragmentId(params.getBackendId(), params.getFragmentId())); + if (fragmentTask == null || !fragmentTask.processReportExecStatus(params)) { + return; + } + TUniqueId queryId = coordinatorContext.queryId; + Status status = new Status(params.status); + // for now, abort the query if we see any error except if the error is cancelled + // and returned_all_results_ is true. + // (UpdateStatus() initiates cancellation, if it hasn't already been initiated) + if (!status.ok()) { + if (coordinatorContext.isEos() && status.isCancelled()) { + LOG.warn("Query {} has returned all results, fragment_id={} instance_id={}, be={}" + + " is reporting failed status {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), + status.toString()); + } else { + LOG.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={}," + + " error message: {}", + DebugUtil.printId(queryId), params.getFragmentId(), + DebugUtil.printId(params.getFragmentInstanceId()), + params.getBackendId(), status.toString()); + coordinatorContext.updateStatusIfOk(status); + } + } + LoadContext loadContext = coordinatorContext.asLoadProcessor().loadContext; + if (params.isSetDeltaUrls()) { + loadContext.updateDeltaUrls(params.getDeltaUrls()); + } + if (params.isSetLoadCounters()) { + loadContext.updateLoadCounters(params.getLoadCounters()); + } + if (params.isSetTrackingUrl()) { + loadContext.updateTrackingUrl(params.getTrackingUrl()); + } + if (params.isSetTxnId()) { + loadContext.updateTransactionId(params.getTxnId()); + } + if (params.isSetLabel()) { + loadContext.updateLabel(params.getLabel()); + } + if (params.isSetExportFiles()) { + loadContext.addExportFiles(params.getExportFiles()); + } + if (params.isSetCommitInfos()) { + loadContext.updateCommitInfos(params.getCommitInfos()); + } + if (params.isSetErrorTabletInfos()) { + loadContext.updateErrorTabletInfos(params.getErrorTabletInfos()); + } + long txnId = loadContext.getTransactionId(); + if (params.isSetHivePartitionUpdates()) { + ((HMSTransaction) Env.getCurrentEnv().getGlobalExternalTransactionInfoMgr().getTxnById(txnId)) + .updateHivePartitionUpdates(params.getHivePartitionUpdates()); + } + if (params.isSetIcebergCommitDatas()) { + ((IcebergTransaction) Env.getCurrentEnv().getGlobalExternalTransactionInfoMgr().getTxnById(txnId)) + .updateIcebergCommitData(params.getIcebergCommitDatas()); + } + + if (fragmentTask.isDone()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Query {} fragment {} is marked done", + DebugUtil.printId(queryId), params.getFragmentId()); + } + latch.get().markedCountDown(params.getFragmentId(), params.getBackendId()); + } + + if (params.isSetLoadedRows() && jobId != -1) { + if (params.isSetFragmentInstanceReports()) { + for (TFragmentInstanceReport report : params.getFragmentInstanceReports()) { + Env.getCurrentEnv().getLoadManager().updateJobProgress( + jobId, params.getBackendId(), params.getQueryId(), report.getFragmentInstanceId(), + report.getLoadedRows(), report.getLoadedBytes(), params.isDone()); + Env.getCurrentEnv().getProgressManager().updateProgress(String.valueOf(jobId), + params.getQueryId(), report.getFragmentInstanceId(), report.getNumFinishedRange()); + } + } else { + Env.getCurrentEnv().getLoadManager().updateJobProgress( + jobId, params.getBackendId(), params.getQueryId(), params.getFragmentInstanceId(), + params.getLoadedRows(), params.getLoadedBytes(), params.isDone()); + Env.getCurrentEnv().getProgressManager().updateProgress(String.valueOf(jobId), + params.getQueryId(), params.getFragmentInstanceId(), params.getFinishedScanRanges()); + } + } + } + + private Map buildBackendFragmentTasks( + SqlPipelineTask executionTask) { + ImmutableMap.Builder backendFragmentTasks + = ImmutableMap.builder(); + for (Entry backendTask : executionTask.getChildrenTasks().entrySet()) { + Long backendId = backendTask.getKey(); + for (Entry fragmentIdToTask : backendTask.getValue() + .getChildrenTasks().entrySet()) { + Integer fragmentId = fragmentIdToTask.getKey(); + SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); + backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); + } + } + return backendFragmentTasks.build(); + } + + /* + * Check the state of backends in needCheckBackendExecStates. + * return true if all of them are OK. Otherwise, return false. + */ + private boolean checkHealthy() { + for (SingleFragmentPipelineTask topFragmentTask : topFragmentTasks) { + if (!topFragmentTask.isBackendHealthy(jobId)) { + long backendId = topFragmentTask.getBackend().getId(); + Status unhealthyStatus = new Status( + TStatusCode.INTERNAL_ERROR, "backend " + backendId + " is down"); + coordinatorContext.updateStatusIfOk(unhealthyStatus); + return false; + } + } + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/MultiFragmentsPipelineTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/MultiFragmentsPipelineTask.java new file mode 100644 index 00000000000000..d30ae2b2cb5352 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/MultiFragmentsPipelineTask.java @@ -0,0 +1,221 @@ +// 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.qe.runtime; + +import org.apache.doris.common.Status; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PCancelPlanFragmentResult; +import org.apache.doris.proto.InternalService.PExecPlanFragmentResult; +import org.apache.doris.proto.InternalService.PExecPlanFragmentStartRequest; +import org.apache.doris.proto.Types; +import org.apache.doris.proto.Types.PUniqueId; +import org.apache.doris.qe.Coordinator; +import org.apache.doris.qe.SimpleScheduler; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.protobuf.ByteString; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +public class MultiFragmentsPipelineTask extends AbstractRuntimeTask { + private static final Logger LOG = LogManager.getLogger(SqlPipelineTask.class); + + // immutable parameters + private final SqlCoordinatorContext coordinatorContext; + private final Backend backend; + private final BackendServiceProxy backendClientProxy; + + // mutable states + + // we will set fragmentsParams and serializeFragments to null after send rpc, to save memory + private ByteString serializeFragments; + private final AtomicBoolean hasCancelled; + private final AtomicBoolean cancelInProcess; + + public MultiFragmentsPipelineTask( + SqlCoordinatorContext coordinatorContext, Backend backend, BackendServiceProxy backendClientProxy, + ByteString serializeFragments, + Map fragmentTasks) { + super(new ChildrenRuntimeTasks<>(fragmentTasks)); + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.backend = Objects.requireNonNull(backend, "backend can not be null"); + this.backendClientProxy = Objects.requireNonNull(backendClientProxy, "backendClientProxy can not be null"); + this.serializeFragments = Objects.requireNonNull( + serializeFragments, "serializeFragments can not be null" + ); + this.hasCancelled = new AtomicBoolean(); + this.cancelInProcess = new AtomicBoolean(); + } + + public Future sendPhaseOneRpc(boolean twoPhaseExecution) { + return execRemoteFragmentsAsync( + backendClientProxy, serializeFragments, backend.getBrpcAddress(), twoPhaseExecution + ); + } + + public Future sendPhaseTwoRpc() { + return execPlanFragmentStartAsync(backendClientProxy, backend.getBrpcAddress()); + } + + @Override + public String toString() { + TNetworkAddress brpcAddress = backend.getBrpcAddress(); + return "MultiFragmentsPipelineTask(Backend " + backend.getId() + + "(" + brpcAddress.getHostname() + ":" + brpcAddress.getPort() + "): [" + + childrenTasks.allTasks() + .stream() + .map(singleFragment -> "F" + singleFragment.getFragmentId()) + .collect(Collectors.joining(", ")) + "])"; + } + + public synchronized void cancelExecute(Status cancelReason) { + TUniqueId queryId = coordinatorContext.queryId; + if (LOG.isDebugEnabled()) { + LOG.debug("cancelRemoteFragments backend: {}, query={}, reason: {}", + backend, DebugUtil.printId(queryId), cancelReason.toString()); + } + + if (this.hasCancelled.get() || this.cancelInProcess.get()) { + LOG.info("Frangment has already been cancelled. Query {} backend: {}", + DebugUtil.printId(queryId), backend); + return; + } + try { + TNetworkAddress brpcAddress = backend.getBrpcAddress(); + try { + ListenableFuture cancelResult = + BackendServiceProxy.getInstance().cancelPipelineXPlanFragmentAsync( + brpcAddress, queryId, cancelReason); + Futures.addCallback(cancelResult, new FutureCallback() { + public void onSuccess(InternalService.PCancelPlanFragmentResult result) { + cancelInProcess.set(false); + if (result.hasStatus()) { + Status status = new Status(result.getStatus()); + if (status.getErrorCode() == TStatusCode.OK) { + hasCancelled.set(true); + } else { + LOG.warn("Failed to cancel query {} backend: {}, reason: {}", + DebugUtil.printId(queryId), backend, status.toString()); + } + } + LOG.warn("Failed to cancel query {} backend: {} reason: {}", + DebugUtil.printId(queryId), backend, "without status"); + } + + public void onFailure(Throwable t) { + cancelInProcess.set(false); + LOG.warn("Failed to cancel query {} backend: {}, reason: {}", + DebugUtil.printId(queryId), backend, cancelReason.toString(), t); + } + }, Coordinator.backendRpcCallbackExecutor); + cancelInProcess.set(true); + } catch (RpcException e) { + LOG.warn("cancel plan fragment get a exception, address={}:{}", brpcAddress.getHostname(), + brpcAddress.getPort()); + SimpleScheduler.addToBlacklist(backend.getId(), e.getMessage()); + } + } catch (Exception e) { + LOG.warn("catch a exception", e); + } + } + + public Backend getBackend() { + return backend; + } + + private Future execRemoteFragmentsAsync( + BackendServiceProxy proxy, ByteString serializedFragments, TNetworkAddress brpcAddr, + boolean twoPhaseExecution) { + Preconditions.checkNotNull(serializedFragments); + try { + return proxy.execPlanFragmentsAsync(brpcAddr, serializedFragments, twoPhaseExecution); + } catch (RpcException e) { + // DO NOT throw exception here, return a complete future with error code, + // so that the following logic will cancel the fragment. + return futureWithException(e); + } finally { + // save memory + this.serializeFragments = null; + } + } + + public Future execPlanFragmentStartAsync( + BackendServiceProxy proxy, TNetworkAddress brpcAddr) { + TUniqueId queryId = coordinatorContext.queryId; + try { + PExecPlanFragmentStartRequest.Builder builder = PExecPlanFragmentStartRequest.newBuilder(); + PUniqueId qid = PUniqueId.newBuilder().setHi(queryId.hi).setLo(queryId.lo).build(); + builder.setQueryId(qid); + return proxy.execPlanFragmentStartAsync(brpcAddr, builder.build()); + } catch (RpcException e) { + // DO NOT throw exception here, return a complete future with error code, + // so that the following logic will cancel the fragment. + return futureWithException(e); + } + } + + private Future futureWithException(RpcException e) { + return new Future() { + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return false; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return true; + } + + @Override + public PExecPlanFragmentResult get() { + PExecPlanFragmentResult result = PExecPlanFragmentResult.newBuilder().setStatus( + Types.PStatus.newBuilder().addErrorMsgs(e.getMessage()) + .setStatusCode(TStatusCode.THRIFT_RPC_ERROR.getValue()).build()).build(); + return result; + } + + @Override + public PExecPlanFragmentResult get(long timeout, TimeUnit unit) { + return get(); + } + }; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java new file mode 100644 index 00000000000000..766f2589d36fd6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/QueryProcessor.java @@ -0,0 +1,195 @@ +// 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.qe.runtime; + +import org.apache.doris.common.Status; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.ResultSink; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.JobProcessor; +import org.apache.doris.qe.ResultReceiver; +import org.apache.doris.qe.RowBatch; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.rpc.RpcException; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TStatusCode; + +import com.google.common.base.Strings; +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.apache.thrift.TException; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.CopyOnWriteArrayList; + +public class QueryProcessor implements JobProcessor { + private static final Logger LOG = LogManager.getLogger(QueryProcessor.class); + + // constant fields + private final long limitRows; + + // mutable field + private Optional sqlPipelineTask; + private final SqlCoordinatorContext coordinatorContext; + private final List runningReceivers; + private int receiverOffset; + private long numReceivedRows; + + public QueryProcessor(SqlCoordinatorContext coordinatorContext, List runningReceivers) { + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.runningReceivers = new CopyOnWriteArrayList<>( + Objects.requireNonNull(runningReceivers, "runningReceivers can not be null") + ); + + this.limitRows = coordinatorContext.fragments.get(coordinatorContext.fragments.size() - 1) + .getPlanRoot() + .getLimit(); + + this.sqlPipelineTask = Optional.empty(); + } + + public static QueryProcessor build(SqlCoordinatorContext coordinatorContext) { + PipelineDistributedPlan topFragment = coordinatorContext.topDistributedPlan; + DataSink topDataSink = coordinatorContext.dataSink; + Boolean enableParallelResultSink; + if (topDataSink instanceof ResultSink) { + enableParallelResultSink = coordinatorContext.queryOptions.isEnableParallelResultSink(); + } else { + enableParallelResultSink = coordinatorContext.queryOptions.isEnableParallelOutfile(); + } + + List topInstances = topFragment.getInstanceJobs(); + List receivers = Lists.newArrayListWithCapacity(topInstances.size()); + Map distinctWorkerJobs = Maps.newLinkedHashMap(); + for (AssignedJob topInstance : topInstances) { + distinctWorkerJobs.putIfAbsent(topInstance.getAssignedWorker().id(), topInstance); + } + + for (AssignedJob topInstance : distinctWorkerJobs.values()) { + DistributedPlanWorker topWorker = topInstance.getAssignedWorker(); + TNetworkAddress execBeAddr = new TNetworkAddress(topWorker.host(), topWorker.brpcPort()); + receivers.add( + new ResultReceiver( + coordinatorContext.queryId, + topInstance.instanceId(), + topWorker.id(), + execBeAddr, + coordinatorContext.timeoutDeadline.get(), + coordinatorContext.connectContext.getSessionVariable().getMaxMsgSizeOfResultReceiver(), + enableParallelResultSink + ) + ); + } + return new QueryProcessor(coordinatorContext, receivers); + } + + @Override + public void setSqlPipelineTask(SqlPipelineTask sqlPipelineTask) { + this.sqlPipelineTask = Optional.ofNullable(sqlPipelineTask); + } + + public boolean isEos() { + return runningReceivers.isEmpty(); + } + + public RowBatch getNext() throws UserException, TException, RpcException { + ResultReceiver receiver = runningReceivers.get(receiverOffset); + Status status = new Status(); + RowBatch resultBatch = receiver.getNext(status); + if (!status.ok()) { + LOG.warn("Query {} coordinator get next fail, {}, need cancel.", + DebugUtil.printId(coordinatorContext.queryId), status.getErrorMsg()); + } + coordinatorContext.updateStatusIfOk(status); + + Status copyStatus = coordinatorContext.readCloneStatus(); + if (!copyStatus.ok()) { + if (Strings.isNullOrEmpty(copyStatus.getErrorMsg())) { + copyStatus.rewriteErrorMsg(); + } + if (copyStatus.isRpcError()) { + throw new RpcException(null, copyStatus.getErrorMsg()); + } else { + String errMsg = copyStatus.getErrorMsg(); + LOG.warn("query failed: {}", errMsg); + throw new UserException(errMsg); + } + } + + if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().dryRunQuery) { + if (resultBatch.isEos()) { + numReceivedRows += resultBatch.getQueryStatistics().getReturnedRows(); + } + } else if (resultBatch.getBatch() != null) { + numReceivedRows += resultBatch.getBatch().getRowsSize(); + } + + if (resultBatch.isEos()) { + runningReceivers.remove(receiver); + if (!runningReceivers.isEmpty()) { + resultBatch.setEos(false); + } + + // if this query is a block query do not cancel. + boolean hasLimit = limitRows > 0; + if (!coordinatorContext.isBlockQuery + && coordinatorContext.instanceNum.get() > 1 + && hasLimit && numReceivedRows >= limitRows) { + if (LOG.isDebugEnabled()) { + LOG.debug("no block query, return num >= limit rows, need cancel"); + } + coordinatorContext.cancelSchedule(new Status(TStatusCode.LIMIT_REACH, "query reach limit")); + } + } + + if (!runningReceivers.isEmpty()) { + receiverOffset = (receiverOffset + 1) % runningReceivers.size(); + } + return resultBatch; + } + + public void cancel(Status cancelReason) { + for (ResultReceiver receiver : runningReceivers) { + receiver.cancel(cancelReason); + } + + this.sqlPipelineTask.ifPresent(sqlPipelineTask -> { + for (MultiFragmentsPipelineTask fragmentsTask : sqlPipelineTask.getChildrenTasks().values()) { + fragmentsTask.cancelExecute(cancelReason); + } + }); + } + + public int getReceiverOffset() { + return receiverOffset; + } + + public long getNumReceivedRows() { + return numReceivedRows; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/RuntimeFiltersThriftBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/RuntimeFiltersThriftBuilder.java new file mode 100644 index 00000000000000..42cf08fb2e3b18 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/RuntimeFiltersThriftBuilder.java @@ -0,0 +1,169 @@ +// 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.qe.runtime; + +import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.BackendWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.RuntimeFilter; +import org.apache.doris.planner.RuntimeFilterId; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TRuntimeFilterParams; +import org.apache.doris.thrift.TRuntimeFilterTargetParams; +import org.apache.doris.thrift.TRuntimeFilterTargetParamsV2; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** RuntimeFiltersThriftBuilder */ +public class RuntimeFiltersThriftBuilder { + public final TNetworkAddress mergeAddress; + + private final List runtimeFilters; + private final AssignedJob mergeInstance; + private final Set broadcastRuntimeFilterIds; + private final Map> ridToTargets; + private final Map ridToBuilderNum; + + private RuntimeFiltersThriftBuilder( + TNetworkAddress mergeAddress, List runtimeFilters, + AssignedJob mergeInstance, Set broadcastRuntimeFilterIds, + Map> ridToTargets, + Map ridToBuilderNum) { + this.mergeAddress = mergeAddress; + this.runtimeFilters = runtimeFilters; + this.mergeInstance = mergeInstance; + this.broadcastRuntimeFilterIds = broadcastRuntimeFilterIds; + this.ridToTargets = ridToTargets; + this.ridToBuilderNum = ridToBuilderNum; + } + + public boolean isMergeRuntimeFilterInstance(AssignedJob instance) { + return mergeInstance == instance; + } + + public void setRuntimeFilterThriftParams(TRuntimeFilterParams runtimeFilterParams) { + for (RuntimeFilter rf : runtimeFilters) { + List targets = ridToTargets.get(rf.getFilterId()); + if (targets == null) { + continue; + } + + if (rf.hasRemoteTargets()) { + Map targetToParams = new LinkedHashMap<>(); + for (RuntimeFilterTarget target : targets) { + TRuntimeFilterTargetParamsV2 targetParams = targetToParams.computeIfAbsent( + target.address, address -> { + TRuntimeFilterTargetParamsV2 params = new TRuntimeFilterTargetParamsV2(); + params.target_fragment_instance_addr = address; + params.target_fragment_instance_ids = new ArrayList<>(); + return params; + }); + + targetParams.target_fragment_instance_ids.add(target.instanceId); + } + + runtimeFilterParams.putToRidToTargetParamv2( + rf.getFilterId().asInt(), new ArrayList<>(targetToParams.values()) + ); + } else { + List targetParams = Lists.newArrayList(); + for (RuntimeFilterTarget target : targets) { + targetParams.add(new TRuntimeFilterTargetParams(target.instanceId, target.address)); + } + runtimeFilterParams.putToRidToTargetParam(rf.getFilterId().asInt(), targetParams); + } + } + for (Map.Entry entry : ridToBuilderNum.entrySet()) { + boolean isBroadcastRuntimeFilter = broadcastRuntimeFilterIds.contains(entry.getKey().asInt()); + int builderNum = isBroadcastRuntimeFilter ? 1 : entry.getValue(); + runtimeFilterParams.putToRuntimeFilterBuilderNum(entry.getKey().asInt(), builderNum); + } + for (RuntimeFilter rf : runtimeFilters) { + runtimeFilterParams.putToRidToRuntimeFilter(rf.getFilterId().asInt(), rf.toThrift()); + } + } + + public static RuntimeFiltersThriftBuilder compute( + List runtimeFilters, List distributedPlans) { + PipelineDistributedPlan topMostPlan = distributedPlans.get(distributedPlans.size() - 1); + AssignedJob mergeInstance = topMostPlan.getInstanceJobs().get(0); + BackendWorker worker = (BackendWorker) mergeInstance.getAssignedWorker(); + TNetworkAddress mergeAddress = new TNetworkAddress(worker.host(), worker.brpcPort()); + + Set broadcastRuntimeFilterIds = runtimeFilters + .stream() + .filter(RuntimeFilter::isBroadcast) + .map(r -> r.getFilterId().asInt()) + .collect(Collectors.toSet()); + + Map> ridToTargetParam = Maps.newLinkedHashMap(); + Map ridToBuilderNum = Maps.newLinkedHashMap(); + for (PipelineDistributedPlan plan : distributedPlans) { + PlanFragment fragment = plan.getFragmentJob().getFragment(); + // Transform to + for (RuntimeFilterId rid : fragment.getTargetRuntimeFilterIds()) { + List targetFragments = + ridToTargetParam.computeIfAbsent(rid, k -> new ArrayList<>()); + for (AssignedJob instanceJob : plan.getInstanceJobs()) { + BackendWorker backendWorker = (BackendWorker) instanceJob.getAssignedWorker(); + Backend backend = backendWorker.getBackend(); + targetFragments.add(new RuntimeFilterTarget( + instanceJob.instanceId(), + new TNetworkAddress(backend.getHost(), backend.getBrpcPort()) + )); + } + } + + for (RuntimeFilterId rid : fragment.getBuilderRuntimeFilterIds()) { + int distinctWorkerNum = (int) plan.getInstanceJobs() + .stream() + .map(AssignedJob::getAssignedWorker) + .map(DistributedPlanWorker::id) + .distinct() + .count(); + ridToBuilderNum.merge(rid, distinctWorkerNum, Integer::sum); + } + } + return new RuntimeFiltersThriftBuilder( + mergeAddress, runtimeFilters, mergeInstance, + broadcastRuntimeFilterIds, ridToTargetParam, ridToBuilderNum + ); + } + + public static class RuntimeFilterTarget { + public final TUniqueId instanceId; + public final TNetworkAddress address; + + public RuntimeFilterTarget(TUniqueId instanceId, TNetworkAddress address) { + this.instanceId = instanceId; + this.address = address; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SingleFragmentPipelineTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SingleFragmentPipelineTask.java new file mode 100644 index 00000000000000..4f0e97e56d8742 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SingleFragmentPipelineTask.java @@ -0,0 +1,99 @@ +// 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.qe.runtime; + +import org.apache.doris.qe.QueryStatisticsItem.FragmentInstanceInfo; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TReportExecStatusParams; +import org.apache.doris.thrift.TUniqueId; + +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.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +public class SingleFragmentPipelineTask extends LeafRuntimeTask { + private static final Logger LOG = LogManager.getLogger(SqlPipelineTask.class); + + // immutable parameters + private final Backend backend; + private final int fragmentId; + private final long lastMissingHeartbeatTime; + private final Set instanceIds; + + // mutate states + private final AtomicBoolean done = new AtomicBoolean(); + + public SingleFragmentPipelineTask(Backend backend, int fragmentId, Set instanceIds) { + this.backend = backend; + this.fragmentId = fragmentId; + this.instanceIds = instanceIds; + this.lastMissingHeartbeatTime = backend.getLastMissingHeartbeatTime(); + } + + // update profile. + // return true if profile is updated. Otherwise, return false. + // Has to use synchronized to ensure there are not concurrent update threads. Or the done + // state maybe update wrong and will lose data. see https://github.com/apache/doris/pull/29802/files. + public boolean processReportExecStatus(TReportExecStatusParams reportExecStatus) { + // The fragment or instance is not finished, not need update + if (!reportExecStatus.done) { + return false; + } + return this.done.compareAndSet(false, true); + } + + public boolean isBackendHealthy(long jobId) { + if (backend.getLastMissingHeartbeatTime() > lastMissingHeartbeatTime && !backend.isAlive()) { + LOG.warn("backend {} is down while joining the coordinator. job id: {}", backend.getId(), jobId); + return false; + } + return true; + } + + public boolean isDone() { + return done.get(); + } + + public Backend getBackend() { + return backend; + } + + public int getFragmentId() { + return fragmentId; + } + + public List buildFragmentInstanceInfo() { + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBePort()); + List infos = Lists.newArrayListWithCapacity(instanceIds.size()); + for (TUniqueId instanceId : instanceIds) { + infos.add( + new FragmentInstanceInfo.Builder() + .address(address) + .fragmentId(String.valueOf(fragmentId)) + .instanceId(instanceId) + .build() + ); + } + return infos; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTask.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTask.java new file mode 100644 index 00000000000000..8cf3f58cfaa2a0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTask.java @@ -0,0 +1,261 @@ +// 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.qe.runtime; + +import org.apache.doris.common.Config; +import org.apache.doris.common.Status; +import org.apache.doris.common.UserException; +import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.metric.MetricRepo; +import org.apache.doris.proto.InternalService.PExecPlanFragmentResult; +import org.apache.doris.qe.SimpleScheduler; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.rpc.RpcException; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TQueryOptions; +import org.apache.doris.thrift.TStatusCode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.joda.time.DateTime; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * SqlPipelineTask. + * + * This class is used to describe which backend process which fragments + */ +public class SqlPipelineTask extends AbstractRuntimeTask { + private static final Logger LOG = LogManager.getLogger(SqlPipelineTask.class); + + // immutable parameters + private final long timeoutDeadline; + private final SqlCoordinatorContext coordinatorContext; + private final BackendServiceProxy backendServiceProxy; + private final Map backendFragmentTasks; + + // mutable states + public SqlPipelineTask( + SqlCoordinatorContext coordinatorContext, + BackendServiceProxy backendServiceProxy, + Map fragmentTasks) { + // insert into stmt need latch to wait finish, but query stmt not need because result receiver can wait finish + super(new ChildrenRuntimeTasks<>(fragmentTasks)); + this.coordinatorContext = Objects.requireNonNull(coordinatorContext, "coordinatorContext can not be null"); + this.backendServiceProxy = Objects.requireNonNull(backendServiceProxy, "backendServiceProxy can not be null"); + this.timeoutDeadline = coordinatorContext.timeoutDeadline.get(); + + // flatten to fragment tasks to quickly index by BackendFragmentId, when receive the report message + ImmutableMap.Builder backendFragmentTasks + = ImmutableMap.builder(); + for (Entry backendTask : fragmentTasks.entrySet()) { + Long backendId = backendTask.getKey(); + for (Entry fragmentIdToTask : backendTask.getValue() + .getChildrenTasks().entrySet()) { + Integer fragmentId = fragmentIdToTask.getKey(); + SingleFragmentPipelineTask fragmentTask = fragmentIdToTask.getValue(); + backendFragmentTasks.put(new BackendFragmentId(backendId, fragmentId), fragmentTask); + } + } + this.backendFragmentTasks = backendFragmentTasks.build(); + } + + @Override + public void execute() throws Exception { + coordinatorContext.withLock(() -> { + sendAndWaitPhaseOneRpc(); + if (coordinatorContext.twoPhaseExecution()) { + sendAndWaitPhaseTwoRpc(); + } + return null; + }); + } + + @Override + public String toString() { + return "SqlPipelineTask(\n" + + childrenTasks.allTasks() + .stream() + .map(multiFragmentsPipelineTask -> " " + multiFragmentsPipelineTask) + .collect(Collectors.joining(",\n")) + + "\n)"; + } + + private void sendAndWaitPhaseOneRpc() throws UserException, RpcException { + List rpcs = Lists.newArrayList(); + for (MultiFragmentsPipelineTask fragmentsTask : childrenTasks.allTasks()) { + rpcs.add(new RpcInfo( + fragmentsTask, + DateTime.now().getMillis(), + fragmentsTask.sendPhaseOneRpc(coordinatorContext.twoPhaseExecution())) + ); + } + Map> rpcPhase1Latency = waitPipelineRpc(rpcs, + timeoutDeadline - System.currentTimeMillis(), "send fragments"); + + coordinatorContext.updateProfileIfPresent(profile -> profile.updateFragmentRpcCount(rpcs.size())); + coordinatorContext.updateProfileIfPresent(SummaryProfile::setFragmentSendPhase1Time); + coordinatorContext.updateProfileIfPresent(profile -> profile.setRpcPhase1Latency(rpcPhase1Latency)); + } + + private void sendAndWaitPhaseTwoRpc() throws RpcException, UserException { + List rpcs = Lists.newArrayList(); + for (MultiFragmentsPipelineTask fragmentTask : childrenTasks.allTasks()) { + rpcs.add(new RpcInfo( + fragmentTask, + DateTime.now().getMillis(), + fragmentTask.sendPhaseTwoRpc()) + ); + } + + Map> rpcPhase2Latency = waitPipelineRpc(rpcs, + timeoutDeadline - System.currentTimeMillis(), "send execution start"); + coordinatorContext.updateProfileIfPresent(profile -> profile.updateFragmentRpcCount(rpcs.size())); + coordinatorContext.updateProfileIfPresent(SummaryProfile::setFragmentSendPhase2Time); + coordinatorContext.updateProfileIfPresent(profile -> profile.setRpcPhase2Latency(rpcPhase2Latency)); + } + + private Map> waitPipelineRpc( + List rpcs, + long leftTimeMs, String operation) throws RpcException, UserException { + TQueryOptions queryOptions = coordinatorContext.queryOptions; + TUniqueId queryId = coordinatorContext.queryId; + + if (leftTimeMs <= 0) { + long currentTimeMillis = System.currentTimeMillis(); + long elapsed = (currentTimeMillis - timeoutDeadline) / 1000 + queryOptions.getExecutionTimeout(); + String msg = String.format( + "timeout before waiting %s rpc, query timeout:%d, already elapsed:%d, left for this:%d", + operation, queryOptions.getExecutionTimeout(), elapsed, leftTimeMs); + LOG.warn("Query {} {}", DebugUtil.printId(queryId), msg); + if (!queryOptions.isSetExecutionTimeout() || !queryOptions.isSetQueryTimeout()) { + LOG.warn("Query {} does not set timeout info, execution timeout: is_set:{}, value:{}" + + ", query timeout: is_set:{}, value: {}, " + + "coordinator timeout deadline {}, cur time millis: {}", + DebugUtil.printId(queryId), + queryOptions.isSetExecutionTimeout(), queryOptions.getExecutionTimeout(), + queryOptions.isSetQueryTimeout(), queryOptions.getQueryTimeout(), + timeoutDeadline, currentTimeMillis); + } + throw new UserException(msg); + } + + // BE -> (RPC latency from FE to BE, Execution latency on bthread, Duration of doing work, RPC latency from BE + // to FE) + Map> beToPrepareLatency = new HashMap<>(); + long timeoutMs = Math.min(leftTimeMs, Config.remote_fragment_exec_timeout_ms); + for (RpcInfo rpc : rpcs) { + TStatusCode code; + String errMsg = null; + Exception exception = null; + + Backend backend = rpc.task.getBackend(); + long beId = backend.getId(); + TNetworkAddress brpcAddress = backend.getBrpcAddress(); + + try { + PExecPlanFragmentResult result = rpc.future.get(timeoutMs, TimeUnit.MILLISECONDS); + long rpcDone = DateTime.now().getMillis(); + beToPrepareLatency.put(brpcAddress, + Lists.newArrayList(result.getReceivedTime() - rpc.startTime, + result.getExecutionTime() - result.getReceivedTime(), + result.getExecutionDoneTime() - result.getExecutionTime(), + rpcDone - result.getExecutionDoneTime())); + code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code == null) { + code = TStatusCode.INTERNAL_ERROR; + } + + if (code != TStatusCode.OK) { + if (!result.getStatus().getErrorMsgsList().isEmpty()) { + errMsg = result.getStatus().getErrorMsgsList().get(0); + } else { + errMsg = operation + " failed. backend id: " + beId; + } + } + } catch (ExecutionException e) { + exception = e; + code = TStatusCode.THRIFT_RPC_ERROR; + backendServiceProxy.removeProxy(brpcAddress); + } catch (InterruptedException e) { + exception = e; + code = TStatusCode.INTERNAL_ERROR; + backendServiceProxy.removeProxy(brpcAddress); + } catch (TimeoutException e) { + exception = e; + errMsg = String.format( + "timeout when waiting for %s rpc, query timeout:%d, left timeout for this operation:%d", + operation, queryOptions.getExecutionTimeout(), timeoutMs / 1000); + LOG.warn("Query {} {}", DebugUtil.printId(queryId), errMsg); + code = TStatusCode.TIMEOUT; + backendServiceProxy.removeProxy(brpcAddress); + } + + if (code != TStatusCode.OK) { + if (exception != null && errMsg == null) { + errMsg = operation + " failed. " + exception.getMessage(); + } + Status cancelStatus = new Status(TStatusCode.INTERNAL_ERROR, errMsg); + coordinatorContext.updateStatusIfOk(cancelStatus); + coordinatorContext.cancelSchedule(cancelStatus); + switch (code) { + case TIMEOUT: + MetricRepo.BE_COUNTER_QUERY_RPC_FAILED.getOrAdd(brpcAddress.hostname) + .increase(1L); + throw new RpcException(brpcAddress.hostname, errMsg, exception); + case THRIFT_RPC_ERROR: + MetricRepo.BE_COUNTER_QUERY_RPC_FAILED.getOrAdd(brpcAddress.hostname) + .increase(1L); + SimpleScheduler.addToBlacklist(beId, errMsg); + throw new RpcException(brpcAddress.hostname, errMsg, exception); + default: + throw new UserException(errMsg, exception); + } + } + } + return beToPrepareLatency; + } + + private static class RpcInfo { + public final MultiFragmentsPipelineTask task; + public final long startTime; + public final Future future; + + public RpcInfo(MultiFragmentsPipelineTask task, long startTime, Future future) { + this.task = task; + this.startTime = startTime; + this.future = future; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTaskBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTaskBuilder.java new file mode 100644 index 00000000000000..f4e953522c37dd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/SqlPipelineTaskBuilder.java @@ -0,0 +1,135 @@ +// 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.qe.runtime; + +import org.apache.doris.common.Pair; +import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.nereids.trees.plans.distribute.worker.BackendWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.qe.protocol.TFastSerializer; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TPipelineFragmentParams; +import org.apache.doris.thrift.TPipelineFragmentParamsList; +import org.apache.doris.thrift.TPipelineInstanceParams; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.Maps; +import com.google.protobuf.ByteString; +import org.apache.thrift.protocol.TCompactProtocol.Factory; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +public class SqlPipelineTaskBuilder { + private SqlCoordinatorContext coordinatorContext; + + public SqlPipelineTaskBuilder(SqlCoordinatorContext coordinatorContext) { + this.coordinatorContext = coordinatorContext; + } + + public static SqlPipelineTask build(SqlCoordinatorContext coordinatorContext, + Map workerToFragmentsParam) { + SqlPipelineTaskBuilder builder = new SqlPipelineTaskBuilder(coordinatorContext); + return builder.buildTask(coordinatorContext, workerToFragmentsParam); + } + + private SqlPipelineTask buildTask(SqlCoordinatorContext coordinatorContext, + Map workerToFragmentsParam) { + BackendServiceProxy backendServiceProxy = BackendServiceProxy.getInstance(); + SqlPipelineTask sqlPipelineTask = new SqlPipelineTask( + coordinatorContext, + backendServiceProxy, + buildMultiFragmentTasks(coordinatorContext, backendServiceProxy, workerToFragmentsParam) + ); + coordinatorContext.getJobProcessor().setSqlPipelineTask(sqlPipelineTask); + return sqlPipelineTask; + } + + private Map buildMultiFragmentTasks( + SqlCoordinatorContext coordinatorContext, BackendServiceProxy backendServiceProxy, + Map workerToFragmentsParam) { + + Map workerToSerializeFragments = serializeFragments(workerToFragmentsParam); + + Map fragmentTasks = Maps.newLinkedHashMap(); + for (Entry kv : + workerToFragmentsParam.entrySet()) { + BackendWorker worker = (BackendWorker) kv.getKey(); + TPipelineFragmentParamsList fragmentParamsList = kv.getValue(); + ByteString serializeFragments = workerToSerializeFragments.get(worker); + + Backend backend = worker.getBackend(); + fragmentTasks.put( + worker.id(), + new MultiFragmentsPipelineTask( + coordinatorContext, + backend, + backendServiceProxy, + serializeFragments, + buildSingleFragmentPipelineTask(backend, fragmentParamsList) + ) + ); + } + return fragmentTasks; + } + + private Map buildSingleFragmentPipelineTask( + Backend backend, TPipelineFragmentParamsList fragmentParamsList) { + Map tasks = Maps.newLinkedHashMap(); + for (TPipelineFragmentParams fragmentParams : fragmentParamsList.getParamsList()) { + int fragmentId = fragmentParams.getFragmentId(); + Set instanceIds = fragmentParams.getLocalParams() + .stream() + .map(TPipelineInstanceParams::getFragmentInstanceId) + .collect(Collectors.toSet()); + tasks.put(fragmentId, new SingleFragmentPipelineTask(backend, fragmentId, instanceIds)); + } + return tasks; + } + + private Map serializeFragments( + Map workerToFragmentsParam) { + + AtomicLong compressedSize = new AtomicLong(0); + Map serializedFragments = workerToFragmentsParam.entrySet() + .parallelStream() + .map(kv -> { + try { + ByteString serializeString = + new TFastSerializer(1024, new Factory()).serialize(kv.getValue()); + return Pair.of(kv.getKey(), serializeString); + } catch (Throwable t) { + throw new IllegalStateException(t.getMessage(), t); + } + }) + .peek(kv -> compressedSize.addAndGet(kv.second.size())) + .collect(Collectors.toMap(Pair::key, Pair::value)); + + coordinatorContext.updateProfileIfPresent( + profile -> profile.updateFragmentCompressedSize(compressedSize.get()) + ); + coordinatorContext.updateProfileIfPresent(SummaryProfile::setFragmentSerializeTime); + + return serializedFragments; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java new file mode 100644 index 00000000000000..0b578266a7af7e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java @@ -0,0 +1,568 @@ +// 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.qe.runtime; + +import org.apache.doris.common.Config; +import org.apache.doris.datasource.FileQueryScanNode; +import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.LocalShuffleAssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanRanges; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedScanBucketOlapTableJob; +import org.apache.doris.nereids.trees.plans.physical.TopnFilter; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.DataStreamSink; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.MultiCastDataSink; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.OlapTableSink; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.planner.SortNode; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SqlCoordinatorContext; +import org.apache.doris.thrift.PaloInternalServiceVersion; +import org.apache.doris.thrift.TDataSinkType; +import org.apache.doris.thrift.TFileScanRangeParams; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TPipelineFragmentParams; +import org.apache.doris.thrift.TPipelineFragmentParamsList; +import org.apache.doris.thrift.TPipelineInstanceParams; +import org.apache.doris.thrift.TPlanFragment; +import org.apache.doris.thrift.TPlanFragmentDestination; +import org.apache.doris.thrift.TQueryOptions; +import org.apache.doris.thrift.TRuntimeFilterParams; +import org.apache.doris.thrift.TScanRangeParams; +import org.apache.doris.thrift.TTopnFilterDesc; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.LinkedHashMultiset; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multiset; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.function.BiConsumer; +import java.util.function.Supplier; + +public class ThriftPlansBuilder { + private static final Logger LOG = LogManager.getLogger(ThriftPlansBuilder.class); + + public static Map plansToThrift( + SqlCoordinatorContext sqlCoordinatorContext) { + + List distributedPlans = sqlCoordinatorContext.distributedPlans; + + // we should set runtime predicate first, then we can use heap sort and to thrift + setRuntimePredicateIfNeed(sqlCoordinatorContext.scanNodes); + + RuntimeFiltersThriftBuilder runtimeFiltersThriftBuilder = RuntimeFiltersThriftBuilder.compute( + sqlCoordinatorContext.runtimeFilters, distributedPlans); + Supplier> topNFilterThriftSupplier + = topNFilterToThrift(sqlCoordinatorContext.topnFilters); + + Multiset workerProcessInstanceNum = computeInstanceNumPerWorker(distributedPlans); + Map fragmentsGroupByWorker = Maps.newLinkedHashMap(); + int currentInstanceIndex = 0; + Map sharedFileScanRangeParams = Maps.newLinkedHashMap(); + for (PipelineDistributedPlan currentFragmentPlan : distributedPlans) { + sharedFileScanRangeParams.putAll(computeFileScanRangeParams(currentFragmentPlan)); + + Map exchangeSenderNum = computeExchangeSenderNum(currentFragmentPlan); + ListMultimap instancesPerWorker + = groupInstancePerWorker(currentFragmentPlan); + Map workerToCurrentFragment = Maps.newLinkedHashMap(); + + for (AssignedJob instanceJob : currentFragmentPlan.getInstanceJobs()) { + TPipelineFragmentParams currentFragmentParam = fragmentToThriftIfAbsent( + currentFragmentPlan, instanceJob, workerToCurrentFragment, + instancesPerWorker, exchangeSenderNum, sharedFileScanRangeParams, + workerProcessInstanceNum, sqlCoordinatorContext); + + TPipelineInstanceParams instanceParam = instanceToThrift( + currentFragmentParam, instanceJob, runtimeFiltersThriftBuilder, + topNFilterThriftSupplier, currentInstanceIndex++ + ); + currentFragmentParam.getLocalParams().add(instanceParam); + } + + // arrange fragments by the same worker, + // so we can merge and send multiple fragment to a backend use one rpc + for (Entry kv : workerToCurrentFragment.entrySet()) { + TPipelineFragmentParamsList fragments = fragmentsGroupByWorker.computeIfAbsent( + kv.getKey(), w -> new TPipelineFragmentParamsList()); + fragments.addToParamsList(kv.getValue()); + } + } + + // backend should initialize fragment from target to source in backend, then + // it can bind the receiver fragment for the sender fragment, but frontend + // compute thrift message from source to fragment, so we need to reverse fragments. + for (DistributedPlanWorker worker : fragmentsGroupByWorker.keySet()) { + Collections.reverse(fragmentsGroupByWorker.get(worker).getParamsList()); + } + + setParamsForOlapTableSink(distributedPlans, fragmentsGroupByWorker); + + // remove redundant params to reduce rpc message size + for (Entry kv : fragmentsGroupByWorker.entrySet()) { + boolean isFirstFragmentInCurrentBackend = true; + for (TPipelineFragmentParams fragmentParams : kv.getValue().getParamsList()) { + if (!isFirstFragmentInCurrentBackend) { + fragmentParams.unsetDescTbl(); + fragmentParams.unsetFileScanParams(); + fragmentParams.unsetCoord(); + fragmentParams.unsetQueryGlobals(); + fragmentParams.unsetResourceInfo(); + fragmentParams.setIsSimplifiedParam(true); + } + isFirstFragmentInCurrentBackend = false; + } + } + return fragmentsGroupByWorker; + } + + private static ListMultimap + groupInstancePerWorker(PipelineDistributedPlan fragmentPlan) { + ListMultimap workerToInstances = ArrayListMultimap.create(); + for (AssignedJob instanceJob : fragmentPlan.getInstanceJobs()) { + workerToInstances.put(instanceJob.getAssignedWorker(), instanceJob); + } + return workerToInstances; + } + + private static void setRuntimePredicateIfNeed(Collection scanNodes) { + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + for (SortNode topnFilterSortNode : scanNode.getTopnFilterSortNodes()) { + topnFilterSortNode.setHasRuntimePredicate(); + } + } + } + } + + private static Supplier> topNFilterToThrift(List topnFilters) { + return Suppliers.memoize(() -> { + if (CollectionUtils.isEmpty(topnFilters)) { + return null; + } + + List filterDescs = new ArrayList<>(topnFilters.size()); + for (TopnFilter topnFilter : topnFilters) { + filterDescs.add(topnFilter.toThrift()); + } + return filterDescs; + }); + } + + private static void setParamsForOlapTableSink(List distributedPlans, + Map fragmentsGroupByWorker) { + int numBackendsWithSink = 0; + for (PipelineDistributedPlan distributedPlan : distributedPlans) { + PlanFragment fragment = distributedPlan.getFragmentJob().getFragment(); + if (fragment.getSink() instanceof OlapTableSink) { + numBackendsWithSink += (int) distributedPlan.getInstanceJobs() + .stream() + .map(AssignedJob::getAssignedWorker) + .distinct() + .count(); + } + } + + for (Entry kv : fragmentsGroupByWorker.entrySet()) { + TPipelineFragmentParamsList fragments = kv.getValue(); + for (TPipelineFragmentParams fragmentParams : fragments.getParamsList()) { + if (fragmentParams.getFragment().getOutputSink().getType() == TDataSinkType.OLAP_TABLE_SINK) { + int loadStreamPerNode = 1; + if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable() != null) { + loadStreamPerNode = ConnectContext.get().getSessionVariable().getLoadStreamPerNode(); + } + fragmentParams.setLoadStreamPerNode(loadStreamPerNode); + fragmentParams.setTotalLoadStreams(numBackendsWithSink * loadStreamPerNode); + fragmentParams.setNumLocalSink(fragmentParams.getLocalParams().size()); + LOG.info("num local sink for backend {} is {}", fragmentParams.getBackendId(), + fragmentParams.getNumLocalSink()); + } + } + } + } + + private static Multiset computeInstanceNumPerWorker( + List distributedPlans) { + Multiset workerCounter = LinkedHashMultiset.create(); + for (PipelineDistributedPlan distributedPlan : distributedPlans) { + for (AssignedJob instanceJob : distributedPlan.getInstanceJobs()) { + workerCounter.add(instanceJob.getAssignedWorker()); + } + } + return workerCounter; + } + + private static Map computeExchangeSenderNum(PipelineDistributedPlan distributedPlan) { + Map senderNum = Maps.newLinkedHashMap(); + for (Entry kv : distributedPlan.getInputs().entries()) { + ExchangeNode exchangeNode = kv.getKey(); + PipelineDistributedPlan childPlan = (PipelineDistributedPlan) kv.getValue(); + senderNum.merge(exchangeNode.getId().asInt(), childPlan.getInstanceJobs().size(), Integer::sum); + } + return senderNum; + } + + private static void setMultiCastDestinationThrift(PipelineDistributedPlan fragmentPlan) { + MultiCastDataSink multiCastDataSink = (MultiCastDataSink) fragmentPlan.getFragmentJob().getFragment().getSink(); + List> destinationList = multiCastDataSink.getDestinations(); + + List dataStreamSinks = multiCastDataSink.getDataStreamSinks(); + for (int i = 0; i < dataStreamSinks.size(); i++) { + DataStreamSink realSink = dataStreamSinks.get(i); + List destinations = destinationList.get(i); + for (Entry> kv : fragmentPlan.getDestinations().entrySet()) { + DataSink sink = kv.getKey(); + if (sink == realSink) { + List destInstances = kv.getValue(); + for (AssignedJob destInstance : destInstances) { + destinations.add(instanceToDestination(destInstance)); + } + break; + } + } + } + } + + private static List nonMultiCastDestinationToThrift(PipelineDistributedPlan plan) { + Map> destinationsMapping = plan.getDestinations(); + List destinations = Lists.newArrayList(); + if (!destinationsMapping.isEmpty()) { + List destinationJobs = destinationsMapping.entrySet().iterator().next().getValue(); + for (AssignedJob destinationJob : destinationJobs) { + destinations.add(instanceToDestination(destinationJob)); + } + } + return destinations; + } + + private static TPlanFragmentDestination instanceToDestination(AssignedJob instance) { + DistributedPlanWorker worker = instance.getAssignedWorker(); + String host = worker.host(); + int port = worker.port(); + int brpcPort = worker.brpcPort(); + + TPlanFragmentDestination destination = new TPlanFragmentDestination(); + destination.setServer(new TNetworkAddress(host, port)); + destination.setBrpcServer(new TNetworkAddress(host, brpcPort)); + destination.setFragmentInstanceId(instance.instanceId()); + return destination; + } + + private static TPipelineFragmentParams fragmentToThriftIfAbsent( + PipelineDistributedPlan fragmentPlan, AssignedJob assignedJob, + Map workerToFragmentParams, + ListMultimap instancesPerWorker, + Map exchangeSenderNum, + Map fileScanRangeParamsMap, + Multiset workerProcessInstanceNum, + SqlCoordinatorContext coordinatorContext) { + DistributedPlanWorker worker = assignedJob.getAssignedWorker(); + return workerToFragmentParams.computeIfAbsent(worker, w -> { + PlanFragment fragment = fragmentPlan.getFragmentJob().getFragment(); + ConnectContext connectContext = coordinatorContext.connectContext; + + TPipelineFragmentParams params = new TPipelineFragmentParams(); + params.setIsNereids(true); + params.setBackendId(worker.id()); + params.setProtocolVersion(PaloInternalServiceVersion.V1); + params.setDescTbl(coordinatorContext.descriptorTable); + params.setQueryId(coordinatorContext.queryId); + params.setFragmentId(fragment.getFragmentId().asInt()); + + // Each tParam will set the total number of Fragments that need to be executed on the same BE, + // and the BE will determine whether all Fragments have been executed based on this information. + // Notice. load fragment has a small probability that FragmentNumOnHost is 0, for unknown reasons. + params.setFragmentNumOnHost(workerProcessInstanceNum.count(worker)); + + params.setNeedWaitExecutionTrigger(coordinatorContext.twoPhaseExecution()); + params.setPerExchNumSenders(exchangeSenderNum); + + List nonMultiCastDestinations; + if (fragment.getSink() instanceof MultiCastDataSink) { + nonMultiCastDestinations = Lists.newArrayList(); + setMultiCastDestinationThrift(fragmentPlan); + } else { + nonMultiCastDestinations = nonMultiCastDestinationToThrift(fragmentPlan); + } + params.setDestinations(nonMultiCastDestinations); + + int instanceNumInThisFragment = fragmentPlan.getInstanceJobs().size(); + params.setNumSenders(instanceNumInThisFragment); + params.setTotalInstances(instanceNumInThisFragment); + + params.setCoord(coordinatorContext.coordinatorAddress); + params.setCurrentConnectFe(coordinatorContext.directConnectFrontendAddress.get()); + params.setQueryGlobals(coordinatorContext.queryGlobals); + params.setQueryOptions(new TQueryOptions(coordinatorContext.queryOptions)); + long memLimit = coordinatorContext.queryOptions.getMemLimit(); + // update memory limit for colocate join + if (connectContext != null + && !connectContext.getSessionVariable().isDisableColocatePlan() + && fragment.hasColocatePlanNode()) { + int rate = Math.min(Config.query_colocate_join_memory_limit_penalty_factor, instanceNumInThisFragment); + memLimit = coordinatorContext.queryOptions.getMemLimit() / rate; + } + params.getQueryOptions().setMemLimit(memLimit); + + params.setSendQueryStatisticsWithEveryBatch(fragment.isTransferQueryStatisticsWithEveryBatch()); + + TPlanFragment planThrift = fragment.toThrift(); + planThrift.query_cache_param = fragment.queryCacheParam; + params.setFragment(planThrift); + params.setLocalParams(Lists.newArrayList()); + params.setWorkloadGroups(coordinatorContext.getWorkloadGroups()); + + params.setFileScanParams(fileScanRangeParamsMap); + + if (fragmentPlan.getFragmentJob() instanceof UnassignedScanBucketOlapTableJob) { + int bucketNum = ((UnassignedScanBucketOlapTableJob) fragmentPlan.getFragmentJob()) + .getOlapScanNodes() + .get(0) + .getBucketNum(); + params.setNumBuckets(bucketNum); + } + + List instances = instancesPerWorker.get(worker); + Map instanceToIndex = instanceToIndex(instances); + + // local shuffle params: bucket_seq_to_instance_idx and shuffle_idx_to_instance_idx + params.setBucketSeqToInstanceIdx(computeBucketIdToInstanceId(fragmentPlan, w, instanceToIndex)); + params.setShuffleIdxToInstanceIdx(computeDestIdToInstanceId(fragmentPlan, w, instanceToIndex)); + return params; + }); + } + + private static Map instanceToIndex(List instances) { + Map instanceToIndex = Maps.newLinkedHashMap(); + for (int instanceIndex = 0; instanceIndex < instances.size(); instanceIndex++) { + instanceToIndex.put(instances.get(instanceIndex), instanceIndex); + } + + return instanceToIndex; + } + + private static Map computeFileScanRangeParams( + PipelineDistributedPlan distributedPlan) { + // scan node id -> TFileScanRangeParams + Map fileScanRangeParamsMap = Maps.newLinkedHashMap(); + for (ScanNode scanNode : distributedPlan.getFragmentJob().getScanNodes()) { + if (scanNode instanceof FileQueryScanNode) { + TFileScanRangeParams fileScanRangeParams = ((FileQueryScanNode) scanNode).getFileScanRangeParams(); + fileScanRangeParamsMap.put(scanNode.getId().asInt(), fileScanRangeParams); + } + } + + return fileScanRangeParamsMap; + } + + private static TPipelineInstanceParams instanceToThrift( + TPipelineFragmentParams currentFragmentParam, AssignedJob instance, + RuntimeFiltersThriftBuilder runtimeFiltersThriftBuilder, + Supplier> topNFilterThriftSupplier, int currentInstanceNum) { + TPipelineInstanceParams instanceParam = new TPipelineInstanceParams(); + instanceParam.setFragmentInstanceId(instance.instanceId()); + setScanSourceParam(currentFragmentParam, instance, instanceParam); + + instanceParam.setSenderId(instance.indexInUnassignedJob()); + instanceParam.setBackendNum(currentInstanceNum); + instanceParam.setRuntimeFilterParams(new TRuntimeFilterParams()); + + instanceParam.setTopnFilterDescs(topNFilterThriftSupplier.get()); + + // set for runtime filter + TRuntimeFilterParams runtimeFilterParams = new TRuntimeFilterParams(); + runtimeFilterParams.setRuntimeFilterMergeAddr(runtimeFiltersThriftBuilder.mergeAddress); + instanceParam.setRuntimeFilterParams(runtimeFilterParams); + if (runtimeFiltersThriftBuilder.isMergeRuntimeFilterInstance(instance)) { + runtimeFiltersThriftBuilder.setRuntimeFilterThriftParams(runtimeFilterParams); + } + return instanceParam; + } + + private static void setScanSourceParam( + TPipelineFragmentParams currentFragmentParam, AssignedJob instance, + TPipelineInstanceParams instanceParams) { + + boolean isLocalShuffle = instance instanceof LocalShuffleAssignedJob; + if (isLocalShuffle && ((LocalShuffleAssignedJob) instance).receiveDataFromLocal) { + // save thrift rpc message size, don't need perNodeScanRanges and perNodeSharedScans, + // but the perNodeScanRanges is required rpc field + instanceParams.setPerNodeScanRanges(Maps.newLinkedHashMap()); + return; + } + + ScanSource scanSource = instance.getScanSource(); + PerNodeScanParams scanParams; + if (scanSource instanceof BucketScanSource) { + scanParams = computeBucketScanSourceParam((BucketScanSource) scanSource); + } else { + scanParams = computeDefaultScanSourceParam((DefaultScanSource) scanSource); + } + // perNodeScanRanges is required + instanceParams.setPerNodeScanRanges(scanParams.perNodeScanRanges); + + if (isLocalShuffle) { + // a fragment in a backend only enable local shuffle once for the first local shuffle instance, + // because we just skip set scan params for LocalShuffleAssignedJob.receiveDataFromLocal == true + ignoreDataDistribution(currentFragmentParam); + } + } + + // local shuffle has two functions: + // 1. use 10 scan instances -> local shuffle -> 10 agg instances, this function can balance data in agg + // 2. use 1 scan instance -> local shuffle -> 10 agg, this function is ignore_data_distribution, + // it can add parallel in agg + private static void ignoreDataDistribution(TPipelineFragmentParams currentFragmentParam) { + // `parallel_instances == 1` is the switch of ignore_data_distribution, + // and backend will use 1 instance to scan a little data, and local shuffle to + // # SessionVariable.parallel_pipeline_task_num instances to increment parallel + currentFragmentParam.setParallelInstances(1); + } + + private static PerNodeScanParams computeDefaultScanSourceParam(DefaultScanSource defaultScanSource) { + Map> perNodeScanRanges = Maps.newLinkedHashMap(); + Map perNodeSharedScans = Maps.newLinkedHashMap(); + for (Entry kv : defaultScanSource.scanNodeToScanRanges.entrySet()) { + int scanNodeId = kv.getKey().getId().asInt(); + perNodeScanRanges.put(scanNodeId, kv.getValue().params); + perNodeSharedScans.put(scanNodeId, true); + } + + return new PerNodeScanParams(perNodeScanRanges, perNodeSharedScans); + } + + private static PerNodeScanParams computeBucketScanSourceParam(BucketScanSource bucketScanSource) { + Map> perNodeScanRanges = Maps.newLinkedHashMap(); + Map perNodeSharedScans = Maps.newLinkedHashMap(); + for (Entry> kv : + bucketScanSource.bucketIndexToScanNodeToTablets.entrySet()) { + Map scanNodeToRanges = kv.getValue(); + for (Entry kv2 : scanNodeToRanges.entrySet()) { + int scanNodeId = kv2.getKey().getId().asInt(); + List scanRanges = perNodeScanRanges.computeIfAbsent(scanNodeId, ArrayList::new); + scanRanges.addAll(kv2.getValue().params); + perNodeSharedScans.put(scanNodeId, true); + } + } + return new PerNodeScanParams(perNodeScanRanges, perNodeSharedScans); + } + + private static Map computeBucketIdToInstanceId( + PipelineDistributedPlan receivePlan, DistributedPlanWorker worker, + Map instanceToIndex) { + List instanceJobs = receivePlan.getInstanceJobs(); + if (instanceJobs.isEmpty() || !(instanceJobs.get(0).getScanSource() instanceof BucketScanSource)) { + // bucket_seq_to_instance_id is optional, so we can return null to save memory + return null; + } + + Map bucketIdToInstanceId = Maps.newLinkedHashMap(); + for (AssignedJob instanceJob : instanceJobs) { + if (instanceJob.getAssignedWorker().id() != worker.id()) { + continue; + } + if (instanceJob instanceof LocalShuffleAssignedJob + && ((LocalShuffleAssignedJob) instanceJob).receiveDataFromLocal) { + continue; + } + Integer instanceIndex = instanceToIndex.get(instanceJob); + BucketScanSource bucketScanSource = (BucketScanSource) instanceJob.getScanSource(); + for (Integer bucketIndex : bucketScanSource.bucketIndexToScanNodeToTablets.keySet()) { + bucketIdToInstanceId.put(bucketIndex, instanceIndex); + } + } + return bucketIdToInstanceId; + } + + private static Map computeDestIdToInstanceId( + PipelineDistributedPlan receivePlan, DistributedPlanWorker worker, + Map instanceToIndex) { + if (receivePlan.getInputs().isEmpty()) { + // shuffle_idx_to_index_id is required + return Maps.newLinkedHashMap(); + } + + Map destIdToInstanceId = Maps.newLinkedHashMap(); + filterInstancesWhichReceiveDataFromRemote( + receivePlan, worker, + (instanceJob, destId) -> destIdToInstanceId.put(destId, instanceToIndex.get(instanceJob)) + ); + return destIdToInstanceId; + } + + private static void filterInstancesWhichReceiveDataFromRemote( + PipelineDistributedPlan receivePlan, DistributedPlanWorker filterWorker, + BiConsumer computeFn) { + + // current only support all input plans have same destination with same order, + // so we can get first input plan to compute shuffle index to instance id + Set> exchangeToChildPlanSet = receivePlan.getInputs().entries(); + if (exchangeToChildPlanSet.isEmpty()) { + return; + } + Entry exchangeToChildPlan = exchangeToChildPlanSet.iterator().next(); + ExchangeNode linkNode = exchangeToChildPlan.getKey(); + PipelineDistributedPlan firstInputPlan = (PipelineDistributedPlan) exchangeToChildPlan.getValue(); + Map> sinkToDestInstances = firstInputPlan.getDestinations(); + for (Entry> kv : sinkToDestInstances.entrySet()) { + DataSink senderSink = kv.getKey(); + if (senderSink.getExchNodeId().asInt() == linkNode.getId().asInt()) { + for (int destId = 0; destId < kv.getValue().size(); destId++) { + AssignedJob assignedJob = kv.getValue().get(destId); + if (assignedJob.getAssignedWorker().id() == filterWorker.id()) { + computeFn.accept(assignedJob, destId); + } + } + break; + } + } + } + + private static class PerNodeScanParams { + Map> perNodeScanRanges; + Map perNodeSharedScans; + + public PerNodeScanParams(Map> perNodeScanRanges, + Map perNodeSharedScans) { + this.perNodeScanRanges = perNodeScanRanges; + this.perNodeSharedScans = perNodeSharedScans; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index 97a06176fef04d..41b7b79ad9ff76 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -235,7 +235,7 @@ public Future execPlanFragmentsAsync(TN } public Future execPlanFragmentStartAsync(TNetworkAddress address, - PExecPlanFragmentStartRequest request) throws TException, RpcException { + PExecPlanFragmentStartRequest request) throws RpcException { try { final BackendServiceClient client = getProxy(address); return client.execPlanFragmentStartAsync(request); 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 a1cac53599556d..01eb92b9be3f40 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 @@ -667,7 +667,7 @@ private static TFetchSchemaTableDataResult queriesMetadataResult(TSchemaTableReq trow.addToColumnValue(new TCell().setLongVal(-1)); } - List tgroupList = queryInfo.getCoord().gettWorkloadGroups(); + List tgroupList = queryInfo.getCoord().getTWorkloadGroups(); if (tgroupList != null && tgroupList.size() == 1) { trow.addToColumnValue(new TCell().setLongVal(tgroupList.get(0).id)); } else { diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query10.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query10.groovy index bfe96f54652eca..f9c6d80b84f704 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query10.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query10.groovy @@ -24,6 +24,7 @@ suite("query10") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query11.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query11.groovy index 96053d3f5186a8..a99c0fb2844909 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query11.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query11.groovy @@ -24,6 +24,7 @@ suite("query11") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query12.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query12.groovy index 50c6332ab10b0a..db173ea3d81a45 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query12.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query12.groovy @@ -24,6 +24,7 @@ suite("query12") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query13.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query13.groovy index 452e4fe6c5c3a8..e6850d187a5c6d 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query13.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query13.groovy @@ -24,6 +24,7 @@ suite("query13") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query14.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query14.groovy index e1adcf8f7355f7..d508d7ba1a586c 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query14.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query14.groovy @@ -24,6 +24,7 @@ suite("query14") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query15.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query15.groovy index 3f53835b3a8f28..e244c34686be9e 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query15.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query15.groovy @@ -24,6 +24,7 @@ suite("query15") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query16.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query16.groovy index b5823a551ee298..90e5de42bdb5aa 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query16.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query16.groovy @@ -24,6 +24,7 @@ suite("query16") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query17.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query17.groovy index f6e30e0a51df4e..779f982558c90f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query17.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query17.groovy @@ -24,6 +24,7 @@ suite("query17") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query18.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query18.groovy index 8a567608bcd551..ae0ea359c1d09c 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query18.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query18.groovy @@ -24,6 +24,7 @@ suite("query18") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query19.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query19.groovy index 112a8d889278ba..70aa3e4f4131da 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query19.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query19.groovy @@ -24,6 +24,7 @@ suite("query19") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query2.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query2.groovy index 6557c5b2fe1d67..50ffb28aeb41b7 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query2.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query2.groovy @@ -24,6 +24,7 @@ suite("query2") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query20.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query20.groovy index 46ad1fc099ee88..419e9fc593ada4 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query20.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query20.groovy @@ -24,6 +24,7 @@ suite("query20") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query21.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query21.groovy index 65da4964236784..9c50c722a73ac3 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query21.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query21.groovy @@ -24,6 +24,7 @@ suite("query21") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'SET enable_fold_constant_by_be = false' //plan shape will be different sql 'set exec_mem_limit=21G' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query22.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query22.groovy index c98156b7ea98ce..5ef9a0bbbbcce2 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query22.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query22.groovy @@ -24,6 +24,7 @@ suite("query22") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query23.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query23.groovy index 4f146d3049c409..117dea7b35b5c9 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query23.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query23.groovy @@ -22,6 +22,7 @@ suite("query23") { multi_sql """ use ${db}; set enable_nereids_planner=true; + set enable_nereids_distribute_planner=false; set enable_fallback_to_original_planner=false; set exec_mem_limit=21G; set be_number_for_test=3; diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query25.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query25.groovy index df41dfc657a098..19d4ef8443f952 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query25.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query25.groovy @@ -24,6 +24,7 @@ suite("query25") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query26.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query26.groovy index e17e3068a2e1f1..c71bb92aac72eb 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query26.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query26.groovy @@ -24,6 +24,7 @@ suite("query26") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query27.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query27.groovy index 326056a0be7b61..ec5f4ac2f1ad93 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query27.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query27.groovy @@ -24,6 +24,7 @@ suite("query27") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query28.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query28.groovy index 15bc959e5e2774..8c43bbaf31144a 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query28.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query28.groovy @@ -24,6 +24,7 @@ suite("query28") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query29.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query29.groovy index b2184e353d8fdc..f8c7f94cf07170 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query29.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query29.groovy @@ -24,6 +24,7 @@ suite("query29") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query3.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query3.groovy index 88cef8e1149c31..db9d172dfb4cfa 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query3.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query3.groovy @@ -24,6 +24,7 @@ suite("query3") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query30.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query30.groovy index 7d49e734418c94..0d89beaab9f0c2 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query30.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query30.groovy @@ -24,6 +24,7 @@ suite("query30") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query31.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query31.groovy index 124e5780f7cc89..06d3717ca0600f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query31.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query31.groovy @@ -24,6 +24,7 @@ suite("query31") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query32.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query32.groovy index 2ed929bc09a85b..2d45b854ee61bf 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query32.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query32.groovy @@ -22,6 +22,7 @@ suite("query32") { multi_sql """ use ${db}; set enable_nereids_planner=true; + set enable_nereids_distribute_planner=false; set enable_fallback_to_original_planner=false; set exec_mem_limit=21G; set be_number_for_test=3; diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query34.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query34.groovy index a02f193f73bc78..79972f33bb7ee0 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query34.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query34.groovy @@ -24,6 +24,7 @@ suite("query34") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query36.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query36.groovy index 679cc544729616..580f025d597eee 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query36.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query36.groovy @@ -24,6 +24,7 @@ suite("query36") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query37.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query37.groovy index ab42a262c99724..c28de3ae6e6a13 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query37.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query37.groovy @@ -24,6 +24,7 @@ suite("query37") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query38.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query38.groovy index d76741c55f7c75..06e3857fb30f3f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query38.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query38.groovy @@ -22,6 +22,7 @@ suite("query38") { multi_sql """ use ${db}; set enable_nereids_planner=true; + set enable_nereids_distribute_planner=false; set enable_fallback_to_original_planner=false; set exec_mem_limit=21G; set be_number_for_test=3; diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query39.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query39.groovy index b9bdc2a49210e0..1a6d2fa5e2ee4a 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query39.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query39.groovy @@ -24,6 +24,7 @@ suite("query39") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query4.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query4.groovy index d8f0f196e4f4d5..2ed84768d45544 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query4.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query4.groovy @@ -24,6 +24,7 @@ suite("query4") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query40.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query40.groovy index d1bd18556fea5a..9daa738d8bf42b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query40.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query40.groovy @@ -24,6 +24,7 @@ suite("query40") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query41.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query41.groovy index 1a772e4381c18b..8cd49d695fc938 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query41.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query41.groovy @@ -24,6 +24,7 @@ suite("query41") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query42.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query42.groovy index b8108863abac71..9a368e763ecc1b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query42.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query42.groovy @@ -24,6 +24,7 @@ suite("query42") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query43.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query43.groovy index 2a7fa4182fb445..ec20d53433519e 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query43.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query43.groovy @@ -24,6 +24,7 @@ suite("query43") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query44.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query44.groovy index dc5ff670d46f92..a6804718503b46 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query44.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query44.groovy @@ -24,6 +24,7 @@ suite("query44") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query45.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query45.groovy index bfd50a87c7c381..3a22a33b595309 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query45.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query45.groovy @@ -24,6 +24,7 @@ suite("query45") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query46.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query46.groovy index add34e3b2b5e1c..3f255f8c6be36b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query46.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query46.groovy @@ -24,6 +24,7 @@ suite("query46") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query47.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query47.groovy index cee7e5b8063ecd..200125adf9e845 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query47.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query47.groovy @@ -24,6 +24,7 @@ suite("query47") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query48.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query48.groovy index a037b912f27ab7..f44e3f3ac46e8f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query48.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query48.groovy @@ -24,6 +24,7 @@ suite("query48") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query49.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query49.groovy index a55f97573559ec..148c9f6fbfa7a2 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query49.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query49.groovy @@ -24,6 +24,7 @@ suite("query49") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query5.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query5.groovy index cc6724b9fe4fbb..3232b380eade74 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query5.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query5.groovy @@ -24,6 +24,7 @@ suite("query5") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query50.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query50.groovy index 3fd2148402984a..77771941874530 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query50.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query50.groovy @@ -24,6 +24,7 @@ suite("query50") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query51.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query51.groovy index 28407a6f99d61e..a8e9d1b590d3cd 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query51.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query51.groovy @@ -24,6 +24,7 @@ suite("query51") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query52.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query52.groovy index d5c5cc65d773eb..4735f2cc493044 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query52.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query52.groovy @@ -24,6 +24,7 @@ suite("query52") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query53.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query53.groovy index 57817314076135..e44ccfed048967 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query53.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query53.groovy @@ -24,6 +24,7 @@ suite("query53") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query54.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query54.groovy index 089fec36a58995..8918b599a3d4ca 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query54.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query54.groovy @@ -24,6 +24,7 @@ suite("query54") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query55.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query55.groovy index aa504a78dc96f9..976dfaa71e1380 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query55.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query55.groovy @@ -24,6 +24,7 @@ suite("query55") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query56.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query56.groovy index d0986a54082146..a311cf8771405b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query56.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query56.groovy @@ -24,6 +24,7 @@ suite("query56") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query57.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query57.groovy index cbe22b62ab37b2..d344c350cc5593 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query57.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query57.groovy @@ -24,6 +24,7 @@ suite("query57") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query58.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query58.groovy index d08b3811f0b826..27238e51637d2e 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query58.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query58.groovy @@ -24,6 +24,7 @@ suite("query58") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query59.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query59.groovy index 0d00b2673d7077..c7df47907a3eb7 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query59.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query59.groovy @@ -24,6 +24,7 @@ suite("query59") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query6.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query6.groovy index 2af04b0f098ed4..ff89e5b12b621f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query6.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query6.groovy @@ -24,6 +24,7 @@ suite("query6") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query60.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query60.groovy index bdc3d839a61575..ddcd041e001be0 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query60.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query60.groovy @@ -24,6 +24,7 @@ suite("query60") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query61.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query61.groovy index 12d07343dc778a..916b9b686c82d2 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query61.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query61.groovy @@ -24,6 +24,7 @@ suite("query61") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query62.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query62.groovy index 5d9185aafc9ede..6d143dabec5686 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query62.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query62.groovy @@ -24,6 +24,7 @@ suite("query62") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query63.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query63.groovy index e0ed0dfc9f4284..d056b177e2bb9b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query63.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query63.groovy @@ -24,6 +24,7 @@ suite("query63") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query65.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query65.groovy index 350f39e995d4e5..a174c367df2993 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query65.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query65.groovy @@ -24,6 +24,7 @@ suite("query65") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query66.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query66.groovy index 5fbfc112265637..c55b1e7298d844 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query66.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query66.groovy @@ -24,6 +24,7 @@ suite("query66") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query68.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query68.groovy index 165e59030045dd..fc2debddbf87b6 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query68.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query68.groovy @@ -24,6 +24,7 @@ suite("query68") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query69.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query69.groovy index e8decd915ffc84..8037d49d7579a6 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query69.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query69.groovy @@ -24,6 +24,7 @@ suite("query69") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query7.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query7.groovy index dc80a626155ebe..205a9f191250df 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query7.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query7.groovy @@ -24,6 +24,7 @@ suite("query7") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query70.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query70.groovy index cf29c62a109ffa..d3751ca94af8cb 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query70.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query70.groovy @@ -24,6 +24,7 @@ suite("query70") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query71.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query71.groovy index fa6cfc8fa6f304..2e5e1ccb85b6ee 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query71.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query71.groovy @@ -24,6 +24,7 @@ suite("query71") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query73.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query73.groovy index fe2ac1c2c6444d..c48eb2e748875e 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query73.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query73.groovy @@ -24,6 +24,7 @@ suite("query73") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query74.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query74.groovy index 551c2b40427da2..944e97c1ef1f57 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query74.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query74.groovy @@ -24,6 +24,7 @@ suite("query74") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query75.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query75.groovy index 4671d4b2cb4525..b91fbb54fa8194 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query75.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query75.groovy @@ -24,6 +24,7 @@ suite("query75") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query76.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query76.groovy index 74f10342af9e1c..c83d452b0545e6 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query76.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query76.groovy @@ -24,6 +24,7 @@ suite("query76") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query77.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query77.groovy index 234e29fc890097..6100f7d42a8a85 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query77.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query77.groovy @@ -24,6 +24,7 @@ suite("query77") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query79.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query79.groovy index b199f9dd8e6126..d4a0140eb6f866 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query79.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query79.groovy @@ -24,6 +24,7 @@ suite("query79") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query8.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query8.groovy index 3847a5791f5267..ae733a9d404418 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query8.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query8.groovy @@ -24,6 +24,7 @@ suite("query8") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query80.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query80.groovy index 8a6cf98923dabc..57fd016217d51b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query80.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query80.groovy @@ -24,6 +24,7 @@ suite("query80") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query81.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query81.groovy index e82f2160399186..8fffbf8cb13a59 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query81.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query81.groovy @@ -24,6 +24,7 @@ suite("query81") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query82.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query82.groovy index 556ff81e6914bf..89bbce3fcaf47f 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query82.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query82.groovy @@ -24,6 +24,7 @@ suite("query82") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query84.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query84.groovy index 77f2be66c31c99..9e08f9874de012 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query84.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query84.groovy @@ -24,6 +24,7 @@ suite("query84") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query85.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query85.groovy index 634cadcdce33e9..ae21cca9e688c7 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query85.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query85.groovy @@ -24,6 +24,7 @@ suite("query85") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query86.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query86.groovy index e9040241c60f09..1756297cd401e4 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query86.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query86.groovy @@ -24,6 +24,7 @@ suite("query86") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query87.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query87.groovy index 63c347e1fc9fda..664c930f03f32b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query87.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query87.groovy @@ -24,6 +24,7 @@ suite("query87") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query88.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query88.groovy index 3d8016e08a9f13..5517225e390144 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query88.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query88.groovy @@ -24,6 +24,7 @@ suite("query88") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query89.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query89.groovy index 10331673c2d79d..50a936d91ec2cf 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query89.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query89.groovy @@ -24,6 +24,7 @@ suite("query89") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query9.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query9.groovy index e178b8830252fc..63359a154eda53 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query9.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query9.groovy @@ -24,6 +24,7 @@ suite("query9") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query90.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query90.groovy index 677bc502d810f9..3e8dd027c8a0fd 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query90.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query90.groovy @@ -24,6 +24,7 @@ suite("query90") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query91.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query91.groovy index dc5dd5fbcde699..cf8c30e1f87466 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query91.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query91.groovy @@ -24,6 +24,7 @@ suite("query91") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query92.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query92.groovy index 92d14918a4cd16..211dda8ed160df 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query92.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query92.groovy @@ -24,6 +24,7 @@ suite("query92") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query93.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query93.groovy index 06dbb93ba58268..513447b1bc0f5d 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query93.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query93.groovy @@ -24,6 +24,7 @@ suite("query93") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query94.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query94.groovy index 0dd0b234f7d50f..a0ca4799a1520b 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query94.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query94.groovy @@ -24,6 +24,7 @@ suite("query94") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query95.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query95.groovy index 41a160948ea6c7..3e32c96597d5dc 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query95.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query95.groovy @@ -24,6 +24,7 @@ suite("query95") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query96.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query96.groovy index db0de98dd96180..df1fe47d768905 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query96.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query96.groovy @@ -24,6 +24,7 @@ suite("query96") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query97.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query97.groovy index 100ad5d9758a73..49ac842916cc83 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query97.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query97.groovy @@ -22,6 +22,7 @@ suite("query97") { multi_sql """ use ${db}; set enable_nereids_planner=true; + set enable_nereids_distribute_planner=false; set enable_fallback_to_original_planner=false; set exec_mem_limit=21G; set be_number_for_test=3; diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query98.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query98.groovy index c53d64b5ddbfde..348379bef2af16 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query98.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query98.groovy @@ -24,6 +24,7 @@ suite("query98") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_hint_tpcds_p0/shape/query99.groovy b/regression-test/suites/nereids_hint_tpcds_p0/shape/query99.groovy index b6d762ac7465ec..b1f2ac7ab311f9 100644 --- a/regression-test/suites/nereids_hint_tpcds_p0/shape/query99.groovy +++ b/regression-test/suites/nereids_hint_tpcds_p0/shape/query99.groovy @@ -24,6 +24,7 @@ suite("query99") { } sql "use ${db}" sql 'set enable_nereids_planner=true' + sql 'set enable_nereids_distribute_planner=false' sql 'set enable_fallback_to_original_planner=false' sql 'set exec_mem_limit=21G' sql 'set be_number_for_test=3' diff --git a/regression-test/suites/nereids_p0/join/test_outer_join.groovy b/regression-test/suites/nereids_p0/join/test_outer_join.groovy index 0f901c070a9bc8..9264f3fc57959d 100644 --- a/regression-test/suites/nereids_p0/join/test_outer_join.groovy +++ b/regression-test/suites/nereids_p0/join/test_outer_join.groovy @@ -19,6 +19,7 @@ suite("test_outer_join", "nereids_p0") { sql "SET enable_nereids_planner=true" sql "SET enable_fallback_to_original_planner=false" sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set enable_nereids_distribute_planner=false;" def tbl1 = "test_outer_join1" def tbl2 = "test_outer_join2" diff --git a/regression-test/suites/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.groovy b/regression-test/suites/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.groovy index fc38e3be3372f0..4499a0ede49f70 100644 --- a/regression-test/suites/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eliminate_outer_join/eliminate_outer_join.groovy @@ -25,6 +25,7 @@ suite("eliminate_outer_join") { sql 'set be_number_for_test=3' sql "set enable_parallel_result_sink=false;" sql "set disable_join_reorder=true;" + sql "set enable_nereids_distribute_planner=false;" sql """ DROP TABLE IF EXISTS t diff --git a/regression-test/suites/nereids_rules_p0/filter_push_down/push_down_alias_through_join.groovy b/regression-test/suites/nereids_rules_p0/filter_push_down/push_down_alias_through_join.groovy index c54fd49f47fef8..c4402f0f07d6ce 100644 --- a/regression-test/suites/nereids_rules_p0/filter_push_down/push_down_alias_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/filter_push_down/push_down_alias_through_join.groovy @@ -24,6 +24,7 @@ suite("push_down_alias_through_join") { sql "set disable_join_reorder=true" sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql "set enable_parallel_result_sink=false;" + sql "set enable_nereids_distribute_planner=false;" // Push alias through inner join where condition not use alias qt_pushdown_inner_join""" diff --git a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy index d29bac550fa6e3..b05b47c3461f90 100644 --- a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy +++ b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy @@ -21,6 +21,7 @@ suite("infer_set_operator_distinct") { sql "SET enable_fallback_to_original_planner=false" sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql "set enable_parallel_result_sink=false;" + sql "set enable_nereids_distribute_planner=false;" sql """ DROP TABLE IF EXISTS t1; diff --git a/regression-test/suites/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.groovy b/regression-test/suites/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.groovy index 933fe3400cfb49..3f959c91fdc04e 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_top_n/push_down_top_n_distinct_through_join.groovy @@ -22,6 +22,7 @@ suite("push_down_top_n_distinct_through_join") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" sql "SET disable_join_reorder=true" + sql "set enable_nereids_distribute_planner=false;" sql """ DROP TABLE IF EXISTS table_join; @@ -66,4 +67,4 @@ suite("push_down_top_n_distinct_through_join") { qt_push_down_topn_through_join_data """ select distinct * from (select t1.id from table_join t1 cross join table_join t2) t order by id limit 10; """ -} \ No newline at end of file +} diff --git a/regression-test/suites/query_p0/union/test_union_instance.groovy b/regression-test/suites/query_p0/union/test_union_instance.groovy index 17a0d3ef1dd614..5fc5d0293852e3 100644 --- a/regression-test/suites/query_p0/union/test_union_instance.groovy +++ b/regression-test/suites/query_p0/union/test_union_instance.groovy @@ -63,6 +63,7 @@ suite("test_union_instance") { set parallel_pipeline_task_num=1; set disable_nereids_rules='PRUNE_EMPTY_PARTITION'; + set enable_nereids_distribute_planner=false; """ explain { sql """