From 63145227c20f5a0d4d2815ddd639f99f0a23e25c Mon Sep 17 00:00:00 2001 From: lichi Date: Fri, 21 Nov 2025 15:53:36 +0800 Subject: [PATCH 01/14] [feature](nereids)support unnest function --- .../org/apache/doris/nereids/DorisLexer.g4 | 2 + .../org/apache/doris/nereids/DorisParser.g4 | 14 + .../BuiltinTableGeneratingFunctions.java | 4 +- .../translator/PhysicalPlanTranslator.java | 5 +- .../doris/nereids/jobs/executor/Rewriter.java | 5 + .../nereids/parser/LogicalPlanBuilder.java | 106 +++- .../apache/doris/nereids/rules/RuleType.java | 2 + .../rules/analysis/BindExpression.java | 61 +- .../nereids/rules/analysis/CheckAnalysis.java | 3 +- .../rules/analysis/NormalizeAggregate.java | 50 +- .../LogicalGenerateToPhysicalGenerate.java | 1 + .../nereids/rules/rewrite/MergeGenerates.java | 27 +- .../rewrite/PushDownUnnestInProject.java | 138 +++++ .../trees/copier/LogicalPlanDeepCopier.java | 6 +- .../functions/generator/Unnest.java | 149 +++++ .../TableGeneratingFunctionVisitor.java | 5 + .../trees/plans/logical/LogicalGenerate.java | 50 +- .../plans/physical/PhysicalGenerate.java | 43 +- .../doris/nereids/util/ExpressionUtils.java | 89 ++- .../doris/planner/TableFunctionNode.java | 10 +- gensrc/thrift/PlanNodes.thrift | 1 + .../gen_function/unnest.out | 554 ++++++++++++++++++ .../gen_function/unnest.groovy | 365 ++++++++++++ 23 files changed, 1563 insertions(+), 127 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java create mode 100644 regression-test/data/nereids_function_p0/gen_function/unnest.out create mode 100644 regression-test/suites/nereids_function_p0/gen_function/unnest.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 index 33282dc4a6b624..7e3318c92ec323 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 @@ -390,6 +390,7 @@ OPTIMIZE: 'OPTIMIZE'; OPTIMIZED: 'OPTIMIZED'; OR: 'OR'; ORDER: 'ORDER'; +ORDINALITY: 'ORDINALITY'; OUTER: 'OUTER'; OUTFILE: 'OUTFILE'; OVER: 'OVER'; @@ -563,6 +564,7 @@ UNINSTALL: 'UNINSTALL'; UNION: 'UNION'; UNIQUE: 'UNIQUE'; UNLOCK: 'UNLOCK'; +UNNEST: 'UNNEST'; UNSET: 'UNSET'; UNSIGNED: 'UNSIGNED'; UP: 'UP'; diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d916626420678f..8cee69db614579 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -1356,6 +1356,17 @@ lateralView tableName=identifier AS columnNames+=identifier (COMMA columnNames+=identifier)* ; +unnest: + LATERAL? UNNEST LEFT_PAREN expression (COMMA expression)* RIGHT_PAREN ( + WITH ORDINALITY + )? ( + AS? tableName = identifier ( + LEFT_PAREN columnNames += identifier ( + COMMA columnNames += identifier + )* RIGHT_PAREN + )? + )?; + queryOrganization : sortClause? limitClause? ; @@ -1415,6 +1426,7 @@ relationPrimary (properties=propertyItemList)? RIGHT_PAREN tableAlias #tableValuedFunction | LEFT_PAREN relations RIGHT_PAREN #relationList + | unnest #unnestFunction ; materializedViewName @@ -2122,6 +2134,7 @@ nonReserved | OPEN | OPTIMIZE | OPTIMIZED + | ORDINALITY | PARAMETER | PARSED | PASSWORD @@ -2235,6 +2248,7 @@ nonReserved | TYPES | UNCOMMITTED | UNLOCK + | UNNEST | UNSET | UP | USER diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableGeneratingFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableGeneratingFunctions.java index cec4807c8f202e..1ad8b5ccf4ba8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableGeneratingFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableGeneratingFunctions.java @@ -40,6 +40,7 @@ import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeVariantArray; import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplode; import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplodeOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -78,7 +79,8 @@ public class BuiltinTableGeneratingFunctions implements FunctionHelper { tableGenerating(ExplodeJsonArrayJsonOuter.class, "explode_json_array_json_outer"), tableGenerating(ExplodeVariantArray.class, "explode_variant_array"), tableGenerating(PosExplode.class, "posexplode"), - tableGenerating(PosExplodeOuter.class, "posexplode_outer") + tableGenerating(PosExplodeOuter.class, "posexplode_outer"), + tableGenerating(Unnest.class, "unnest") ); public static final ImmutableSet RETURN_MULTI_COLUMNS_FUNCTIONS = new ImmutableSortedSet.Builder( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 84e7c400269cf3..38e5976dd7349b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -1452,8 +1452,11 @@ public PlanFragment visitPhysicalGenerate(PhysicalGenerate gener .flatMap(List::stream) .map(SlotDescriptor::getId) .collect(Collectors.toList()); + ArrayList conjuncts = generate.getConjuncts().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toCollection(ArrayList::new)); TableFunctionNode tableFunctionNode = new TableFunctionNode(context.nextPlanNodeId(), - currentFragment.getPlanRoot(), tupleDescriptor.getId(), functionCalls, outputSlotIds); + currentFragment.getPlanRoot(), tupleDescriptor.getId(), functionCalls, outputSlotIds, conjuncts); tableFunctionNode.setNereidsId(generate.getId()); context.getNereidsIdToPlanNodeIdMap().put(generate.getId(), tableFunctionNode.getId()); addPlanRoot(currentFragment, tableFunctionNode, generate); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index ed0fa310228a70..38eabe669f9bbf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -146,6 +146,7 @@ import org.apache.doris.nereids.rules.rewrite.PushDownTopNThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushDownTopNThroughUnion; import org.apache.doris.nereids.rules.rewrite.PushDownTopNThroughWindow; +import org.apache.doris.nereids.rules.rewrite.PushDownUnnestInProject; import org.apache.doris.nereids.rules.rewrite.PushDownVectorTopNIntoOlapScan; import org.apache.doris.nereids.rules.rewrite.PushDownVirtualColumnsIntoOlapScan; import org.apache.doris.nereids.rules.rewrite.PushFilterInsideJoin; @@ -535,6 +536,10 @@ public class Rewriter extends AbstractBatchJobExecutor { new SimplifyWindowExpression() ) ), + topic("Push down Unnest", + topDown( + new PushDownUnnestInProject() + )), topic("Rewrite join", // infer not null filter, then push down filter, and then reorder join(cross join to inner join) topDown( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index fe02442ee2aba0..556f45084dceda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -562,6 +562,7 @@ import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.scalar.Array; import org.apache.doris.nereids.trees.expressions.functions.scalar.ArraySlice; import org.apache.doris.nereids.trees.expressions.functions.scalar.Char; @@ -2719,6 +2720,58 @@ public LogicalPlan visitTableValuedFunction(TableValuedFunctionContext ctx) { }); } + @Override + public LogicalPlan visitUnnestFunction(DorisParser.UnnestFunctionContext ctx) { + return withUnnest(ctx.unnest()); + } + + private LogicalPlan withUnnest(DorisParser.UnnestContext ctx) { + String defaultNestedColumnName = "unnest"; + String defaultOrdinalityColumnName = "ordinality"; + List arguments = ctx.expression().stream() + .map(this::typedVisit) + .collect(ImmutableList.toImmutableList()); + boolean needOrdinality = ctx.ORDINALITY() != null; + int size = arguments.size(); + + String generateName = ctx.tableName != null ? ctx.tableName.getText() : defaultNestedColumnName; + // do same thing as later view explode map type, we need to add a project to convert map to struct + int argumentsSize = size + (needOrdinality ? 1 : 0); + List nestedColumnNames = new ArrayList<>(argumentsSize); + int columnNamesSize = ctx.columnNames.size(); + if (!ctx.columnNames.isEmpty()) { + for (int i = 0; i < columnNamesSize; ++i) { + nestedColumnNames.add(ctx.columnNames.get(i).getText()); + } + for (int i = 0; i < size - columnNamesSize; ++i) { + nestedColumnNames.add(defaultNestedColumnName); + } + if (needOrdinality && columnNamesSize < argumentsSize) { + nestedColumnNames.add(defaultOrdinalityColumnName); + } + } else { + if (size == 1) { + nestedColumnNames.add(generateName); + } else { + for (int i = 0; i < size; ++i) { + nestedColumnNames.add(defaultNestedColumnName); + } + } + if (needOrdinality) { + nestedColumnNames.add(defaultOrdinalityColumnName); + } + } + String columnName = nestedColumnNames.get(0); + Unnest unnest = new Unnest(false, needOrdinality, arguments); + // only unnest use LogicalOneRowRelation as LogicalGenerate's child, + // so we can check LogicalGenerate's child to know if it's unnest function + return new LogicalGenerate<>(ImmutableList.of(unnest), + ImmutableList.of(new UnboundSlot(generateName, columnName)), + ImmutableList.of(nestedColumnNames), + new LogicalOneRowRelation(StatementScopeIdGenerator.newRelationId(), + ImmutableList.of(new Alias(Literal.of(0))))); + } + /** * Create a star (i.e. all) expression; this selects all elements (in the specified object). * Both un-targeted (global) and targeted aliases are supported. @@ -4358,13 +4411,30 @@ private LogicalPlan withJoinRelations(LogicalPlan input, RelationContext ctx) { } } if (ids == null) { - last = new LogicalJoin<>(joinType, ExpressionUtils.EMPTY_CONDITION, - condition.map(ExpressionUtils::extractConjunction) - .orElse(ExpressionUtils.EMPTY_CONDITION), - distributeHint, - Optional.empty(), - last, - plan(join.relationPrimary()), null); + LogicalPlan right = plan(join.relationPrimary()); + if (right instanceof LogicalGenerate + && right.child(0) instanceof LogicalOneRowRelation + && ((LogicalGenerate) right).getGenerators().get(0) instanceof Unnest) { + if (joinType.isLeftJoin() || joinType.isInnerJoin() || joinType.isCrossJoin()) { + LogicalGenerate oldRight = (LogicalGenerate) right; + Unnest oldGenerator = (Unnest) oldRight.getGenerators().get(0); + Unnest newGenerator = joinType.isLeftJoin() ? oldGenerator.withOuter(true) : oldGenerator; + last = new LogicalGenerate<>(ImmutableList.of(newGenerator), oldRight.getGeneratorOutput(), + oldRight.getExpandColumnAlias(), condition.map(ExpressionUtils::extractConjunction) + .orElse(ExpressionUtils.EMPTY_CONDITION), last); + } else { + throw new ParseException("The combining JOIN type must be INNER, LEFT or CROSS for UNNEST", + join); + } + } else { + last = new LogicalJoin<>(joinType, ExpressionUtils.EMPTY_CONDITION, + condition.map(ExpressionUtils::extractConjunction) + .orElse(ExpressionUtils.EMPTY_CONDITION), + distributeHint, + Optional.empty(), + last, + right, null); + } } else { last = new LogicalUsingJoin<>(joinType, last, plan(join.relationPrimary()), ids, distributeHint); @@ -4586,15 +4656,19 @@ private LogicalPlan withRelations(LogicalPlan inputPlan, List r for (RelationContext relation : relations) { // build left deep join tree LogicalPlan right = withJoinRelations(visitRelation(relation), relation); - left = (left == null) ? right : - new LogicalJoin<>( - JoinType.CROSS_JOIN, - ExpressionUtils.EMPTY_CONDITION, - ExpressionUtils.EMPTY_CONDITION, - new DistributeHint(DistributeType.NONE), - Optional.empty(), - left, - right, null); + // check if it's unnest + boolean shouldBeParent = right instanceof LogicalGenerate + && right.child(0) instanceof LogicalOneRowRelation; + left = (left == null) ? right + : shouldBeParent ? ((LogicalGenerate) right).withChildren(ImmutableList.of(left)) + : new LogicalJoin<>( + JoinType.CROSS_JOIN, + ExpressionUtils.EMPTY_CONDITION, + ExpressionUtils.EMPTY_CONDITION, + new DistributeHint(DistributeType.NONE), + Optional.empty(), + left, + right, null); // TODO: pivot and lateral view } return left; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index a691009be415d4..34585a0adbd143 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -383,6 +383,8 @@ public enum RuleType { PUSH_DOWN_LIMIT_DISTINCT_THROUGH_JOIN(RuleTypeClass.REWRITE), PUSH_DOWN_LIMIT_DISTINCT_THROUGH_PROJECT_JOIN(RuleTypeClass.REWRITE), PUSH_DOWN_LIMIT_DISTINCT_THROUGH_UNION(RuleTypeClass.REWRITE), + // push down unnest + PUSH_DOWN_UNNEST_IN_PROJECT(RuleTypeClass.REWRITE), // adjust nullable ADJUST_NULLABLE(RuleTypeClass.REWRITE), ADJUST_CONJUNCTS_RETURN_TYPE(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 036f7653e615c1..554661be296b3d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -283,7 +283,7 @@ private LogicalPlan bindGenerate(MatchingContext> ctx) { if (!(boundGenerator instanceof TableGeneratingFunction)) { throw new AnalysisException(boundGenerator.toSql() + " is not a TableGeneratingFunction"); } - Function generator = (Function) boundGenerator; + Function generator = ExpressionUtils.convertUnnest((Function) boundGenerator); boundGenerators.add(generator); Slot boundSlot = new SlotReference(slot.getNameParts().get(1), generator.getDataType(), @@ -312,18 +312,61 @@ boundSlot, new StringLiteral(fields.get(idx).getName())), } } } - LogicalGenerate ret = new LogicalGenerate<>( - boundGenerators.build(), outputSlots.build(), generate.child()); + /* + * SELECT + * id, + * tags + * FROM + * items + * LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + * + * t.tag is unnest's output, so the conjunct t.tag = name may reference slot from child and its own output + * + */ + int conjunctSize = generate.getConjuncts().size(); + List newConjuncts = new ArrayList<>(conjunctSize); + if (conjunctSize > 0) { + List childOutputs = generate.child().getOutput(); + List conjunctsScopeSlots = new ArrayList<>(expandAlias.size() + childOutputs.size()); + for (Alias alias : expandAlias) { + conjunctsScopeSlots.add(alias.toSlot()); + } + + conjunctsScopeSlots.addAll(childOutputs); + Scope conjunctsScope = toScope(cascadesContext, conjunctsScopeSlots); + ExpressionAnalyzer conjunctsAnalyzer = new ExpressionAnalyzer( + generate, conjunctsScope, cascadesContext, true, false); + Map replaceMap = ExpressionUtils.generateReplaceMap(expandAlias); + for (Expression expression : generate.getConjuncts()) { + expression = conjunctsAnalyzer.analyze(expression); + Expression newExpression = expression.rewriteDownShortCircuit( + e -> replaceMap.getOrDefault(e, e)); + newConjuncts.add(newExpression); + } + } + + LogicalGenerate logicalGenerate = new LogicalGenerate<>( + boundGenerators.build(), outputSlots.build(), ImmutableList.of(), newConjuncts, generate.child()); if (!expandAlias.isEmpty()) { + // project should contain: generator.child slot + expandAlias + List allProjectSlots = new ArrayList<>(generate.child().getOutput().size() + + expandAlias.size()); + if (!(generate.child() instanceof LogicalOneRowRelation)) { + // project should contain: generator.child slot + expandAlias except: + allProjectSlots.addAll(generate.child().getOutput().stream() + .map(NamedExpression.class::cast) + .collect(Collectors.toList())); + } else { + // unnest with literal argument as unnest([1,2,3]) + // we should not add LogicalOneRowRelation's output slot in this case + // so do nothing + } // we need a project to deal with explode(map) to struct with field alias - // project should contains: generator.child slot + expandAlias - List allProjectSlots = generate.child().getOutput().stream() - .map(NamedExpression.class::cast) - .collect(Collectors.toList()); allProjectSlots.addAll(expandAlias); - return new LogicalProject<>(allProjectSlots, ret); + return new LogicalProject<>(allProjectSlots, logicalGenerate); + } else { + return logicalGenerate; } - return ret; } private LogicalSetOperation bindSetOperation(LogicalSetOperation setOperation) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java index aae1eb6dfde758..44c9522d902f42 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScalarFunction; import org.apache.doris.nereids.trees.expressions.typecoercion.TypeCheckResult; import org.apache.doris.nereids.trees.plans.Plan; @@ -160,7 +161,7 @@ private void checkUnexpectedExpressionTypes(Plan plan, Class { for (Class type : unexpectedExpressionTypes) { - if (type.isInstance(e)) { + if (type.isInstance(e) && !(e instanceof Unnest)) { throw new AnalysisException(plan.getType() + " can not contains " + type.getSimpleName() + " expression: " + ((Expression) e).toSql()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java index d03709f26686f8..25cf0cbaa1040b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java @@ -39,6 +39,7 @@ import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; import org.apache.doris.nereids.trees.plans.Plan; @@ -162,8 +163,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional aggregateOutput = aggregate.getOutputExpressions(); - Map> aggFuncs = - CollectNonWindowedAggFuncsWithSessionVar.collect(aggregateOutput); + Map> aggFuncs = CollectNonWindowedAggFuncsWithSessionVar + .collect(aggregateOutput); // split agg child as two part // TRUE part 1: need push down itself, if it contains subquery or window expression @@ -178,7 +179,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional aggregate, Optional inputSlots - = arg instanceof OrderExpression ? arg.getInputSlots() : ImmutableList.of(arg); + Collection inputSlots = arg instanceof OrderExpression ? arg.getInputSlots() + : ImmutableList.of(arg); for (Expression input : inputSlots) { if (input instanceof SlotReference) { needPushDownInputs.add(input); @@ -208,8 +210,7 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional needPushSelf = needPushDownSelfExprs.build(); Set needPushInputSlots = ExpressionUtils.getInputSlotSet(needPushDownInputs.build()); - Set existsAlias = - ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); + Set existsAlias = ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); // push down 3 kinds of exprs, these pushed exprs will be used to normalize agg output later // 1. group by exprs @@ -223,12 +224,10 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional pushedGroupByExprs = - bottomSlotContext.pushDownToNamedExpression(groupingByExprs); - Set pushedTrivialAggChildren = - bottomSlotContext.pushDownToNamedExpression(needPushSelf); - Set pushedTrivialAggInputSlots = - bottomSlotContext.pushDownToNamedExpression(needPushInputSlots); + Set pushedGroupByExprs = bottomSlotContext.pushDownToNamedExpression(groupingByExprs); + Set pushedTrivialAggChildren = bottomSlotContext.pushDownToNamedExpression(needPushSelf); + Set pushedTrivialAggInputSlots = bottomSlotContext + .pushDownToNamedExpression(needPushInputSlots); Set bottomProjects = Sets.union(pushedGroupByExprs, Sets.union(pushedTrivialAggChildren, pushedTrivialAggInputSlots)); @@ -244,12 +243,11 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional normalizedGroupExprs = - bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); + List normalizedGroupExprs = bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); // normalize trivial-aggs by bottomProjects - List normalizedAggFuncs = - bottomSlotContext.normalizeToUseSlotRef(SessionVarGuardExpr.getExprWithGuard(aggFuncs)); + List normalizedAggFuncs = bottomSlotContext + .normalizeToUseSlotRef(SessionVarGuardExpr.getExprWithGuard(aggFuncs)); if (normalizedAggFuncs.stream().anyMatch(agg -> !agg.children().isEmpty() && agg.child(0).containsType(AggregateFunction.class))) { throw new AnalysisException( @@ -257,20 +255,19 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional normalizedAggOutputBuilder - = ImmutableList.builderWithExpectedSize(groupingByExprs.size() + normalizedAggFuncs.size()); + ImmutableList.Builder normalizedAggOutputBuilder = ImmutableList + .builderWithExpectedSize(groupingByExprs.size() + normalizedAggFuncs.size()); for (NamedExpression pushedGroupByExpr : pushedGroupByExprs) { normalizedAggOutputBuilder.add(pushedGroupByExpr.toSlot()); } normalizedAggOutputBuilder.addAll( - normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs) - ); + normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs)); // create new agg node ImmutableList normalizedAggOutput = normalizedAggOutputBuilder.build(); @@ -324,8 +321,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional newAggregate = - aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutputBuilder.build(), bottomPlan); + LogicalAggregate newAggregate = aggregate.withNormalized(normalizedGroupExprs, + normalizedAggOutputBuilder.build(), bottomPlan); ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(ctx); LogicalProject project = eliminateGroupByConstant(groupByExprContext, rewriteContext, normalizedGroupExprs, normalizedAggOutput, bottomProjects, aggregate, upperProjects, newAggregate); @@ -341,8 +338,7 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional( ExpressionUtils.replace(having.get().getConjuncts(), project.getAliasToProducer()), - project.child() - ))); + project.child()))); } // after build logical plan, it will not extract window expression from the SELECT lists, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalGenerateToPhysicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalGenerateToPhysicalGenerate.java index 22fd1b2639aff5..154680240b077f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalGenerateToPhysicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalGenerateToPhysicalGenerate.java @@ -30,6 +30,7 @@ public Rule build() { return logicalGenerate().then(generate -> new PhysicalGenerate<>( generate.getGenerators(), generate.getGeneratorOutput(), + generate.getConjuncts(), generate.getLogicalProperties(), generate.child()) ).toRule(RuleType.LOGICAL_GENERATE_TO_PHYSICAL_GENERATE); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeGenerates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeGenerates.java index 19cefeb0400eda..5d55a90f6befab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeGenerates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeGenerates.java @@ -35,18 +35,19 @@ public class MergeGenerates extends OneRewriteRuleFactory { @Override public Rule build() { - return logicalGenerate(logicalGenerate()).then(top -> { - LogicalGenerate bottom = top.child(); - Set topGeneratorSlots = top.getInputSlots(); - if (bottom.getGeneratorOutput().stream().anyMatch(topGeneratorSlots::contains)) { - // top generators use bottom's generator's output, cannot merge. - return top; - } - List generators = Lists.newArrayList(bottom.getGenerators()); - generators.addAll(top.getGenerators()); - List generatorsOutput = Lists.newArrayList(bottom.getGeneratorOutput()); - generatorsOutput.addAll(top.getGeneratorOutput()); - return new LogicalGenerate<>(generators, generatorsOutput, bottom.child()); - }).toRule(RuleType.MERGE_GENERATES); + return logicalGenerate(logicalGenerate().when(generate -> generate.getConjuncts().isEmpty())) + .when(generate -> generate.getConjuncts().isEmpty()).then(top -> { + LogicalGenerate bottom = top.child(); + Set topGeneratorSlots = top.getInputSlots(); + if (bottom.getGeneratorOutput().stream().anyMatch(topGeneratorSlots::contains)) { + // top generators use bottom's generator's output, cannot merge. + return top; + } + List generators = Lists.newArrayList(bottom.getGenerators()); + generators.addAll(top.getGenerators()); + List generatorsOutput = Lists.newArrayList(bottom.getGeneratorOutput()); + generatorsOutput.addAll(top.getGeneratorOutput()); + return new LogicalGenerate<>(generators, generatorsOutput, bottom.child()); + }).toRule(RuleType.MERGE_GENERATES); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java new file mode 100644 index 00000000000000..1708e23332c52b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java @@ -0,0 +1,138 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.Function; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; +import org.apache.doris.nereids.trees.expressions.functions.scalar.StructElement; +import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; +import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Before: + * project(unnest(x) as a, unnest(y) as b) + * + * After: + * + * project(struct_element($c$1, col1) as a, struct_element($c$1, co2) as b) + * │ + * ▼ + * generate(unnest(x, y) as $c$1) + */ +public class PushDownUnnestInProject extends OneRewriteRuleFactory { + + @Override + public Rule build() { + return logicalProject().when(project -> containUnnestFunction(project.getProjects())).then(project -> { + List outputs = project.getProjects(); + Set existedAlias = ExpressionUtils.collect(outputs, Alias.class::isInstance); + List toBePushedDown = ExpressionUtils.collectToList(outputs, Unnest.class::isInstance); + Function newFunction = ExpressionUtils.convertUnnest(validateAndMergeUnnest(toBePushedDown)); + NormalizeToSlot.NormalizeToSlotContext context = + NormalizeToSlot.NormalizeToSlotContext.buildContext(existedAlias, toBePushedDown); + String columnName = ConnectContext.get() != null + ? ConnectContext.get().getStatementContext().generateColumnName() : "expand_cols"; + SlotReference outputSlot = new SlotReference("unnest_temp_table", newFunction.getDataType(), + newFunction.nullable(), ImmutableList.of(columnName)); + List newProjects = new ArrayList<>(toBePushedDown.size()); + Map slotTripletMap = context.getNormalizeToSlotMap(); + if (toBePushedDown.size() > 1) { + // struct_element(#expand_col#k, #k) as #k + // struct_element(#expand_col#v, #v) as #v + List fields = ((StructType) outputSlot.getDataType()).getFields(); + Preconditions.checkState(fields.size() == toBePushedDown.size(), String.format("push down" + + "unnest function has error, function count is %d, pushed down count is %d", + toBePushedDown.size(), fields.size())); + for (int i = 0; i < fields.size(); ++i) { + Slot remainExpr = slotTripletMap.get(toBePushedDown.get(i)).remainExpr; + newProjects.add(new Alias(remainExpr.getExprId(), new StructElement( + outputSlot, new StringLiteral(fields.get(i).getName())), remainExpr.getName())); + } + } else { + Slot remainExpr = slotTripletMap.get(toBePushedDown.get(0)).remainExpr; + newProjects.add(new Alias(remainExpr.getExprId(), outputSlot, remainExpr.getName())); + } + Map replaceMap = ExpressionUtils.generateReplaceMap(newProjects); + List newOutputs = context.normalizeToUseSlotRef(outputs); + return project.withProjectsAndChild(ExpressionUtils.replaceNamedExpressions(newOutputs, replaceMap), + new LogicalGenerate(ImmutableList.of(newFunction), ImmutableList.of(outputSlot), project.child())); + }).toRule(RuleType.PUSH_DOWN_UNNEST_IN_PROJECT); + } + + private boolean containUnnestFunction(List expressions) { + for (NamedExpression expr : expressions) { + if (expr.containsType(Unnest.class)) { + return true; + } + } + return false; + } + + private Unnest validateAndMergeUnnest(List functions) { + // TODO: PG only support ARRAY type, and explode_map and explode_bitmap only support 1 argument in doris + // so we only allow array for now, too + int typeCounter = 0; + int size = functions.size(); + List expressions = new ArrayList<>(size); + for (Unnest unnest : functions) { + if (unnest.children().size() > 1) { + throw new AnalysisException("UNNEST function can have more than one arguments only in from clause"); + } + Expression expr = unnest.child(0); + DataType dataType = expr.getDataType(); + if (dataType.isArrayType()) { + typeCounter += 1; + } else if (dataType.isMapType()) { + typeCounter += 2; + } else if (dataType.isBitmapType()) { + typeCounter += 3; + } + expressions.add(expr); + } + // TODO: remove this after doris support multiple arguments for explode_map and explode_bitmap + if (expressions.size() > 1 && typeCounter != 1 * size) { + throw new AnalysisException("multiple UNNEST functions in same place must have ARRAY argument type"); + } + if (typeCounter == 1 * size || typeCounter == 2 * size || typeCounter == 3 * size) { + return new Unnest(false, false, expressions); + } else { + throw new AnalysisException("UNNEST functions must have same argument type"); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java index 33e32fe92e71bd..d7f133a6e99bde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java @@ -403,7 +403,11 @@ public Plan visitLogicalGenerate(LogicalGenerate generate, DeepC List generatorOutput = generate.getGeneratorOutput().stream() .map(o -> (Slot) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) .collect(ImmutableList.toImmutableList()); - return new LogicalGenerate<>(generators, generatorOutput, generate.getExpandColumnAlias(), child); + List conjuncts = generate.getConjuncts().stream() + .map(p -> ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalGenerate<>(generators, generatorOutput, generate.getExpandColumnAlias(), + conjuncts, child); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java new file mode 100644 index 00000000000000..61ac628640908a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java @@ -0,0 +1,149 @@ +// 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.expressions.functions.generator; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ComputePrecision; +import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.SearchSignature; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.BitmapType; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.MapType; +import org.apache.doris.nereids.types.NullType; +import org.apache.doris.nereids.types.StructField; +import org.apache.doris.nereids.types.StructType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.List; + +/** + * unnest([1, 2, 3]), generate three rows include 1, 2 and 3. + * unnest([1, 2, 3], [4, 5, 6]) generates two columns and three rows + * where the first column contains 1, 2, 3, and the second column contains 4, 5, 6. + */ +public class Unnest extends TableGeneratingFunction implements CustomSignature, ComputePrecision, AlwaysNullable { + + private boolean isOuter; + private boolean needOrdinality; + + /** + * constructor with one argument. + */ + public Unnest(Expression argument) { + this(false, false, ImmutableList.of(argument)); + } + + /** + * constructor with more argument. + */ + public Unnest(boolean isOuter, boolean needOrdinality, List arguments) { + super("unnest", arguments); + this.isOuter = isOuter; + this.needOrdinality = needOrdinality; + } + + /** constructor for withChildren and reuse signature */ + private Unnest(boolean isOuter, boolean needOrdinality, GeneratorFunctionParams functionParams) { + super(functionParams); + this.isOuter = isOuter; + this.needOrdinality = needOrdinality; + } + + /** + * withChildren. + */ + @Override + public Unnest withChildren(List children) { + Preconditions.checkArgument(!children.isEmpty()); + return new Unnest(isOuter, needOrdinality, getFunctionParams(children)); + } + + public Unnest withOuter(boolean isOuter) { + return new Unnest(isOuter, needOrdinality, getFunctionParams(children)); + } + + public boolean isOuter() { + return isOuter; + } + + public boolean needOrdinality() { + return needOrdinality; + } + + @Override + public FunctionSignature computePrecision(FunctionSignature signature) { + return signature; + } + + @Override + public FunctionSignature customSignature() { + DataType childDataType = child(0).getDataType(); + if (childDataType.isArrayType() || childDataType.isNullType()) { + List arguments = new ArrayList<>(children.size()); + ImmutableList.Builder structFields = ImmutableList.builder(); + for (int i = 0; i < children.size(); i++) { + if (children.get(i).getDataType().isNullType()) { + arguments.add(ArrayType.of(NullType.INSTANCE)); + structFields.add( + new StructField("col" + (i + 1), NullType.INSTANCE, true, "")); + } else if (children.get(i).getDataType().isArrayType()) { + structFields.add( + new StructField("col" + (i + 1), + ((ArrayType) (children.get(i)).getDataType()).getItemType(), true, "")); + arguments.add(children.get(i).getDataType()); + } else { + SearchSignature.throwCanNotFoundFunctionException(this.getName(), getArguments()); + } + } + + StructType structType = new StructType(structFields.build()); + if (arguments.size() == 1) { + return FunctionSignature.of(structType.getFields().get(0).getDataType(), arguments); + } + return FunctionSignature.of(structType, arguments); + } else if (childDataType.isBitmapType()) { + return FunctionSignature.ret(BigIntType.INSTANCE).args(BitmapType.INSTANCE); + } else if (childDataType.isMapType()) { + return FunctionSignature.ret(new StructType(ImmutableList.of( + new StructField("col1", ((MapType) child(0).getDataType()).getKeyType(), true, ""), + new StructField("col2", ((MapType) child(0).getDataType()).getValueType(), true, "")))) + .args(child(0).getDataType()); + } else { + SearchSignature.throwCanNotFoundFunctionException(this.getName(), getArguments()); + return null; + } + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitUnnest(this, context); + } + + @Override + public FunctionSignature searchSignature(List signatures) { + return super.searchSignature(signatures); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java index 9fae7c397cada9..2a149d0a391bfc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableGeneratingFunctionVisitor.java @@ -41,6 +41,7 @@ import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplode; import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplodeOuter; import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdtf; /** @@ -144,4 +145,8 @@ default R visitPosExplode(PosExplode posExplode, C context) { default R visitPosExplodeOuter(PosExplodeOuter posExplodeOuter, C context) { return visitTableGeneratingFunction(posExplodeOuter, context); } + + default R visitUnnest(Unnest unnest, C context) { + return visitTableGeneratingFunction(unnest, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java index 974317f1942f03..1f7dc0c7b724df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.Generate; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.PlanUtils; import org.apache.doris.nereids.util.Utils; import com.google.common.base.Preconditions; @@ -38,6 +39,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; /** @@ -50,23 +52,34 @@ public class LogicalGenerate extends LogicalUnary generatorOutput; // mapping with function. private final List> expandColumnAlias; + private final List conjuncts; public LogicalGenerate(List generators, List generatorOutput, CHILD_TYPE child) { - this(generators, generatorOutput, ImmutableList.of(), Optional.empty(), Optional.empty(), child); + this(generators, generatorOutput, ImmutableList.of(), ImmutableList.of(), + Optional.empty(), Optional.empty(), child); } public LogicalGenerate(List generators, List generatorOutput, List> expandColumnAlias, CHILD_TYPE child) { - this(generators, generatorOutput, expandColumnAlias, Optional.empty(), Optional.empty(), child); + this(generators, generatorOutput, expandColumnAlias, ImmutableList.of(), + Optional.empty(), Optional.empty(), child); } public LogicalGenerate(List generators, List generatorOutput, List> expandColumnAlias, + List conjuncts, CHILD_TYPE child) { + this(generators, generatorOutput, expandColumnAlias, conjuncts, + Optional.empty(), Optional.empty(), child); + } + + public LogicalGenerate(List generators, List generatorOutput, List> expandColumnAlias, + List conjuncts, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_GENERATE, groupExpression, logicalProperties, child); this.generators = Utils.fastToImmutableList(generators); this.generatorOutput = Utils.fastToImmutableList(generatorOutput); this.expandColumnAlias = Utils.fastToImmutableList(expandColumnAlias); + this.conjuncts = Utils.fastToImmutableList(conjuncts); } public List getGenerators() { @@ -81,10 +94,14 @@ public List> getExpandColumnAlias() { return expandColumnAlias; } + public List getConjuncts() { + return conjuncts; + } + @Override public LogicalGenerate withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, children.get(0)); + return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, conjuncts, children.get(0)); } @Override @@ -94,7 +111,15 @@ public R accept(PlanVisitor visitor, C context) { @Override public List getExpressions() { - return generators; + return new ImmutableList.Builder() + .addAll(generators) + .addAll(conjuncts) + .build(); + } + + @Override + public Set getInputSlots() { + return PlanUtils.fastGetInputSlots(generators); } /** @@ -106,12 +131,12 @@ public LogicalGenerate withGenerators(List generators) { for (int i = 0; i < generators.size(); i++) { newGeneratorOutput.add(generatorOutput.get(i).withNullable(generators.get(i).nullable())); } - return new LogicalGenerate<>(generators, newGeneratorOutput, expandColumnAlias, child()); + return new LogicalGenerate<>(generators, newGeneratorOutput, expandColumnAlias, conjuncts, child()); } @Override public LogicalGenerate withGroupExpression(Optional groupExpression) { - return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, + return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, conjuncts, groupExpression, Optional.of(getLogicalProperties()), child()); } @@ -119,7 +144,7 @@ public LogicalGenerate withGroupExpression(Optional group public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, + return new LogicalGenerate<>(generators, generatorOutput, expandColumnAlias, conjuncts, groupExpression, logicalProperties, children.get(0)); } @@ -144,6 +169,7 @@ public String toString() { return Utils.toSqlStringSkipNull("LogicalGenerate", "generators", generators, "generatorOutput", generatorOutput, + "conjuncts", conjuncts, "stats", statistics ); } @@ -165,7 +191,10 @@ public String toDigest() { .append(" AS ") .append( expandColumnAlias.get(0).stream().collect(Collectors.joining(", ")) - ); + ) + .append(" WHERE ") + .append(!conjuncts.isEmpty() ? conjuncts.stream().map(Expression::toDigest) + .collect(Collectors.joining(" AND ")) : "TRUE"); return sb.toString(); } @@ -179,12 +208,13 @@ public boolean equals(Object o) { } LogicalGenerate that = (LogicalGenerate) o; return generators.equals(that.generators) - && generatorOutput.equals(that.generatorOutput); + && generatorOutput.equals(that.generatorOutput) + && conjuncts.equals(that.conjuncts); } @Override public int hashCode() { - return Objects.hash(generators, generatorOutput); + return Objects.hash(generators, generatorOutput, conjuncts); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java index 8d6dd5c5c83fc9..b84a264bf45211 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java @@ -45,27 +45,19 @@ public class PhysicalGenerate extends PhysicalUnary generators; private final List generatorOutput; + private final List conjuncts; - public PhysicalGenerate(List generators, List generatorOutput, + public PhysicalGenerate(List generators, List generatorOutput, List conjuncts, LogicalProperties logicalProperties, CHILD_TYPE child) { - this(generators, generatorOutput, Optional.empty(), logicalProperties, child); - } - - /** - * constructor - */ - public PhysicalGenerate(List generators, List generatorOutput, - Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { - super(PlanType.PHYSICAL_GENERATE, groupExpression, logicalProperties, child); - this.generators = ImmutableList.copyOf(Objects.requireNonNull(generators, "predicates can not be null")); - this.generatorOutput = ImmutableList.copyOf(Objects.requireNonNull(generatorOutput, - "generatorOutput can not be null")); + this(generators, generatorOutput, conjuncts, Optional.empty(), logicalProperties, + PhysicalProperties.ANY, null, child); } /** * constructor */ public PhysicalGenerate(List generators, List generatorOutput, + List conjuncts, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { @@ -74,6 +66,7 @@ public PhysicalGenerate(List generators, List generatorOutput, this.generators = ImmutableList.copyOf(Objects.requireNonNull(generators, "predicates can not be null")); this.generatorOutput = ImmutableList.copyOf(Objects.requireNonNull(generatorOutput, "generatorOutput can not be null")); + this.conjuncts = ImmutableList.copyOf(conjuncts); } @Override @@ -91,11 +84,16 @@ public List getExpressions() { return generators; } + public List getConjuncts() { + return conjuncts; + } + @Override public String toString() { return Utils.toSqlString("PhysicalGenerate", "generators", generators, - "generatorOutput", generatorOutput + "generatorOutput", generatorOutput, + "conjuncts", conjuncts ); } @@ -109,7 +107,8 @@ public boolean equals(Object o) { } PhysicalGenerate that = (PhysicalGenerate) o; return generators.equals(that.generators) - && generatorOutput.equals(that.generatorOutput); + && generatorOutput.equals(that.generatorOutput) + && conjuncts.equals(that.conjuncts); } @Override @@ -125,28 +124,28 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalGenerate withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new PhysicalGenerate<>(generators, generatorOutput, groupExpression, + return new PhysicalGenerate<>(generators, generatorOutput, conjuncts, groupExpression, getLogicalProperties(), physicalProperties, statistics, children.get(0)); } @Override public PhysicalGenerate withGroupExpression(Optional groupExpression) { - return new PhysicalGenerate<>(generators, generatorOutput, - groupExpression, getLogicalProperties(), child()); + return new PhysicalGenerate<>(generators, generatorOutput, conjuncts, + groupExpression, getLogicalProperties(), PhysicalProperties.ANY, null, child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); - return new PhysicalGenerate<>(generators, generatorOutput, - groupExpression, logicalProperties.get(), children.get(0)); + return new PhysicalGenerate<>(generators, generatorOutput, conjuncts, + groupExpression, logicalProperties.get(), PhysicalProperties.ANY, null, children.get(0)); } @Override public PhysicalGenerate withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalGenerate<>(generators, generatorOutput, + return new PhysicalGenerate<>(generators, generatorOutput, conjuncts, Optional.empty(), getLogicalProperties(), physicalProperties, statistics, child()); } @@ -161,7 +160,7 @@ public List computeOutput() { @Override public PhysicalGenerate resetLogicalProperties() { - return new PhysicalGenerate<>(generators, generatorOutput, + return new PhysicalGenerate<>(generators, generatorOutput, conjuncts, Optional.empty(), null, physicalProperties, statistics, child()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 17e5a45b38927e..a2d25337982dc6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -58,6 +58,15 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Max; import org.apache.doris.nereids.trees.expressions.functions.agg.Min; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.generator.Explode; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeBitmap; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeBitmapOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeMap; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeMapOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplode; +import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplodeOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.scalar.If; import org.apache.doris.nereids.trees.expressions.functions.scalar.NullIf; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl; @@ -74,6 +83,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; +import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.VariantType; import org.apache.doris.nereids.types.coercion.NumericType; import org.apache.doris.qe.ConnectContext; @@ -332,8 +342,8 @@ public static List shuttleExpressionWithLineage(List replaceNamedExpressions(List - e instanceof UniqueFunction ? ((UniqueFunction) e).withIgnoreUniqueId(ignoreUniqueId) : e); + return expression.rewriteDownShortCircuit( + e -> e instanceof UniqueFunction ? ((UniqueFunction) e).withIgnoreUniqueId(ignoreUniqueId) : e); } public static List rewriteDownShortCircuit( @@ -672,10 +682,9 @@ public static boolean canInferNotNullForMarkSlot(Expression predicate, Expressio * the mark slot can be non-nullable boolean * and in semi join, we can safely change the mark conjunct to hash conjunct */ - ImmutableList literals = - ImmutableList.of(NullLiteral.BOOLEAN_INSTANCE, BooleanLiteral.FALSE); - List markJoinSlotReferenceList = - new ArrayList<>((predicate.collect(MarkJoinSlotReference.class::isInstance))); + ImmutableList literals = ImmutableList.of(NullLiteral.BOOLEAN_INSTANCE, BooleanLiteral.FALSE); + List markJoinSlotReferenceList = new ArrayList<>( + (predicate.collect(MarkJoinSlotReference.class::isInstance))); int markSlotSize = markJoinSlotReferenceList.size(); int maxMarkSlotCount = 4; // if the conjunct has mark slot, and maximum 4 mark slots(for performance) @@ -702,8 +711,7 @@ public static boolean canInferNotNullForMarkSlot(Expression predicate, Expressio } Expression evalResult = FoldConstantRule.evaluate( ExpressionUtils.replace(predicate, replaceMap), - ctx - ); + ctx); if (evalResult.equals(BooleanLiteral.TRUE)) { if (meetNullOrFalse) { @@ -742,8 +750,7 @@ public static Set inferNotNullSlots(Set predicates, CascadesCo replaceMap.put(slot, nullLiteral); Expression evalExpr = FoldConstantRule.evaluate( ExpressionUtils.replace(predicate, replaceMap), - new ExpressionRewriteContext(cascadesContext) - ); + new ExpressionRewriteContext(cascadesContext)); if (evalExpr.isNullLiteral() || BooleanLiteral.FALSE.equals(evalExpr)) { notNullSlots.add(slot); } @@ -937,7 +944,8 @@ public static Set mutableCollect(List expressions, public static List collectAll(Collection expressions, Predicate> predicate) { switch (expressions.size()) { - case 0: return ImmutableList.of(); + case 0: + return ImmutableList.of(); default: { ImmutableList.Builder result = ImmutableList.builder(); for (Expression expr : expressions) { @@ -1046,14 +1054,13 @@ public static Expression getSingleNumericSlotOrExpressionCoveredByCast(Expressio */ public static boolean checkSlotConstant(Slot slot, Set predicates) { return predicates.stream().anyMatch(predicate -> { - if (predicate instanceof EqualTo) { - EqualTo equalTo = (EqualTo) predicate; - return (equalTo.left() instanceof Literal && equalTo.right().equals(slot)) - || (equalTo.right() instanceof Literal && equalTo.left().equals(slot)); - } - return false; - } - ); + if (predicate instanceof EqualTo) { + EqualTo equalTo = (EqualTo) predicate; + return (equalTo.left() instanceof Literal && equalTo.right().equals(slot)) + || (equalTo.right() instanceof Literal && equalTo.left().equals(slot)); + } + return false; + }); } /** @@ -1171,8 +1178,7 @@ public static Literal analyzeAndFoldToLiteral(ConnectContext ctx, Expression exp } ExpressionRewriteContext context = new ExpressionRewriteContext(cascadesContext); ExpressionRuleExecutor executor = new ExpressionRuleExecutor(ImmutableList.of( - ExpressionRewrite.bottomUp(ReplaceVariableByLiteral.INSTANCE) - )); + ExpressionRewrite.bottomUp(ReplaceVariableByLiteral.INSTANCE))); Expression rewrittenExpression = executor.rewrite(analyzedExpr, context); Expression foldExpression = FoldConstantRule.evaluate(rewrittenExpression, context); if (foldExpression instanceof Literal) { @@ -1247,8 +1253,8 @@ public static boolean containsCaseWhenLikeType(Expression expression) { public static Optional> getCaseWhenLikeBranchResults(Expression expression) { if (expression instanceof CaseWhen) { CaseWhen caseWhen = (CaseWhen) expression; - ImmutableList.Builder builder - = ImmutableList.builderWithExpectedSize(caseWhen.getWhenClauses().size() + 1); + ImmutableList.Builder builder = ImmutableList + .builderWithExpectedSize(caseWhen.getWhenClauses().size() + 1); for (WhenClause whenClause : caseWhen.getWhenClauses()) { builder.add(whenClause.getResult()); } @@ -1300,4 +1306,37 @@ public static boolean containUniqueFunctionExistMultiple(Collection args = unnest.getArguments(); + if (dataType.isArrayType()) { + Expression[] arrayArgs = args.toArray(new Expression[0]); + return unnest.isOuter() + ? unnest.needOrdinality() ? new PosExplodeOuter(args.get(0)) : new ExplodeOuter(arrayArgs) + : unnest.needOrdinality() ? new PosExplode(args.get(0)) : new Explode(arrayArgs); + } else { + if (unnest.needOrdinality()) { + throw new AnalysisException(String.format("only ARRAY support WITH ORDINALITY," + + " but argument's type is %s", dataType)); + } + if (dataType.isMapType()) { + return unnest.isOuter() ? new ExplodeMapOuter(args.get(0)) : new ExplodeMap(args.get(0)); + } else if (dataType.isBitmapType()) { + return unnest.isOuter() ? new ExplodeBitmapOuter(args.get(0)) : new ExplodeBitmap(args.get(0)); + } else { + throw new AnalysisException(String.format("UNNEST function doesn't support %s argument type, " + + "please try to use lateral view and explode_* function set instead", dataType.toSql())); + } + } + } else { + return function; + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java index 1a0ab65ddc72b6..a5099999366238 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java @@ -37,9 +37,10 @@ public class TableFunctionNode extends PlanNode { // The output slot ids of TableFunctionNode // Only the slot whose id is in this list will be output by TableFunctionNode private List outputSlotIds = Lists.newArrayList(); + private List expandConjuncts = Lists.newArrayList(); public TableFunctionNode(PlanNodeId id, PlanNode inputNode, TupleId lateralViewTupleId, - ArrayList fnCallExprList, List outputSlotIds) { + ArrayList fnCallExprList, List outputSlotIds, List expandConjuncts) { super(id, "TABLE FUNCTION NODE"); if (inputNode.outputTupleDesc != null) { tupleIds.add(inputNode.outputTupleDesc.getId()); @@ -56,6 +57,7 @@ public TableFunctionNode(PlanNodeId id, PlanNode inputNode, TupleId lateralViewT this.fnCallExprList = fnCallExprList; this.outputSlotIds = outputSlotIds; this.children.add(inputNode); + this.expandConjuncts.addAll(expandConjuncts); } public void setOutputSlotIds(List outputSlotIds) { @@ -88,6 +90,11 @@ public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { } output.append("\n"); + if (!expandConjuncts.isEmpty()) { + output.append(prefix).append("expand conjuncts: ").append( + getExplainString(expandConjuncts)).append("\n"); + } + if (!conjuncts.isEmpty()) { output.append(prefix).append("PREDICATES: ").append( getExplainString(conjuncts)).append("\n"); @@ -101,6 +108,7 @@ protected void toThrift(TPlanNode msg) { msg.node_type = TPlanNodeType.TABLE_FUNCTION_NODE; msg.table_function_node = new TTableFunctionNode(); msg.table_function_node.setFnCallExprList(Expr.treesToThrift(fnCallExprList)); + msg.table_function_node.setExpandConjuncts(Expr.treesToThrift(expandConjuncts)); for (SlotId slotId : outputSlotIds) { msg.table_function_node.addToOutputSlotIds(slotId.asInt()); } diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index 8eea8f078db367..d24097e557c413 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -1266,6 +1266,7 @@ struct TOlapRewriteNode { struct TTableFunctionNode { 1: optional list fnCallExprList 2: optional list outputSlotIds + 3: optional list expand_conjuncts } // This contains all of the information computed by the plan as part of the resource diff --git a/regression-test/data/nereids_function_p0/gen_function/unnest.out b/regression-test/data/nereids_function_p0/gen_function/unnest.out new file mode 100644 index 00000000000000..5553db4a89bb2f --- /dev/null +++ b/regression-test/data/nereids_function_p0/gen_function/unnest.out @@ -0,0 +1,554 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql_unnest_bitmap -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_bitmap_notnull -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_outer_bitmap -- +\N \N +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_outer_bitmap_notnull -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_array -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_array_notnull -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_outer_array -- +\N \N +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_outer_array_notnull -- +0 1 +1 2 +10 11 +11 12 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 + +-- !sql_unnest_map -- +0 1 \N +0 2 1 +0 3 2 +1 1 \N +1 2 2 +1 3 1 +10 1 \N +10 2 1 +10 3 2 +11 1 \N +11 2 2 +11 3 1 +2 1 \N +2 2 3 +2 3 0 +3 1 \N +3 2 4 +3 3 9 +4 1 \N +4 2 5 +4 3 8 +5 1 \N +5 2 6 +5 3 7 +6 1 \N +6 2 7 +6 3 6 +7 1 \N +7 2 8 +7 3 5 +8 1 \N +8 2 9 +8 3 4 +9 1 \N +9 2 0 +9 3 3 + +-- !sql_unnest_map_notnull -- +0 1 \N +0 2 1 +0 3 2 +1 1 \N +1 2 2 +1 3 1 +10 1 \N +10 2 1 +10 3 2 +11 1 \N +11 2 2 +11 3 1 +2 1 \N +2 2 3 +2 3 0 +3 1 \N +3 2 4 +3 3 9 +4 1 \N +4 2 5 +4 3 8 +5 1 \N +5 2 6 +5 3 7 +6 1 \N +6 2 7 +6 3 6 +7 1 \N +7 2 8 +7 3 5 +8 1 \N +8 2 9 +8 3 4 +9 1 \N +9 2 0 +9 3 3 + +-- !sql_unnest_outer_map -- +\N \N \N +0 1 \N +0 2 1 +0 3 2 +1 1 \N +1 2 2 +1 3 1 +10 1 \N +10 2 1 +10 3 2 +11 1 \N +11 2 2 +11 3 1 +2 1 \N +2 2 3 +2 3 0 +3 1 \N +3 2 4 +3 3 9 +4 1 \N +4 2 5 +4 3 8 +5 1 \N +5 2 6 +5 3 7 +6 1 \N +6 2 7 +6 3 6 +7 1 \N +7 2 8 +7 3 5 +8 1 \N +8 2 9 +8 3 4 +9 1 \N +9 2 0 +9 3 3 + +-- !sql_unnest_outer_map_notnull -- +0 1 \N +0 2 1 +0 3 2 +1 1 \N +1 2 2 +1 3 1 +10 1 \N +10 2 1 +10 3 2 +11 1 \N +11 2 2 +11 3 1 +2 1 \N +2 2 3 +2 3 0 +3 1 \N +3 2 4 +3 3 9 +4 1 \N +4 2 5 +4 3 8 +5 1 \N +5 2 6 +5 3 7 +6 1 \N +6 2 7 +6 3 6 +7 1 \N +7 2 8 +7 3 5 +8 1 \N +8 2 9 +8 3 4 +9 1 \N +9 2 0 +9 3 3 + +-- !sql_unnest_literal_from_1 -- +1 +2 +3 + +-- !sql_unnest_literal_from_2 -- +\N 22 +1 11 +2 \N +3 \N + +-- !sql_unnest_literal_from_nested -- +\N +[1, 2] +[3] +[4, 5, 6] + +-- !sql_unnest_literal_from_hybrid -- +\N \N +[1, 2] hi +[3] hello +[4, 5, 6] \N + +-- !sql_unnest_literal_from_multiple -- +1 11 +1 22 +2 11 +2 22 +3 11 +3 22 + +-- !sql_unnest_literal_select_1 -- +1 +2 +3 + +-- !sql_unnest_literal_select_nested -- +\N +[1, 2] +[3] +[4, 5, 6] + +-- !sql_unnest_literal_select_multiple -- +1 11 +2 22 +3 \N + +-- !sql_grouping_set -- +\N \N 5 7500.00 +\N \N 5 8800.00 +\N \N 5 9800.00 +\N Keyboard 5 9800.00 +\N Laptop 5 7500.00 +\N Mouse 5 8800.00 +East \N 4 7200.00 +East Keyboard 2 3600.00 +East Laptop 2 3600.00 +North \N 5 6900.00 +North Laptop 3 3900.00 +North Mouse 2 3000.00 +South \N 5 11200.00 +South Keyboard 3 6200.00 +South Mouse 2 5000.00 +West \N 1 800.00 +West Mouse 1 800.00 + +-- !sql_cube -- +\N \N \N 15 26100.00 +\N \N \N 5 7500.00 +\N \N \N 5 8800.00 +\N \N \N 5 9800.00 +\N \N 2023 15 26100.00 +\N \N 2023 5 7500.00 +\N \N 2023 5 8800.00 +\N \N 2023 5 9800.00 +\N Keyboard \N 1 1800.00 +\N Keyboard \N 1 2500.00 +\N Keyboard \N 3 5500.00 +\N Keyboard \N 5 9800.00 +\N Keyboard 2023 1 1800.00 +\N Keyboard 2023 1 2500.00 +\N Keyboard 2023 3 5500.00 +\N Keyboard 2023 5 9800.00 +\N Laptop \N 1 1500.00 +\N Laptop \N 1 1800.00 +\N Laptop \N 3 4200.00 +\N Laptop \N 5 7500.00 +\N Laptop 2023 1 1500.00 +\N Laptop 2023 1 1800.00 +\N Laptop 2023 3 4200.00 +\N Laptop 2023 5 7500.00 +\N Mouse \N 1 1500.00 +\N Mouse \N 1 2500.00 +\N Mouse \N 3 4800.00 +\N Mouse \N 5 8800.00 +\N Mouse 2023 1 1500.00 +\N Mouse 2023 1 2500.00 +\N Mouse 2023 3 4800.00 +\N Mouse 2023 5 8800.00 +East \N \N 2 3600.00 +East \N \N 2 3600.00 +East \N \N 4 7200.00 +East \N 2023 2 3600.00 +East \N 2023 2 3600.00 +East \N 2023 4 7200.00 +East Keyboard \N 1 1800.00 +East Keyboard \N 1 1800.00 +East Keyboard \N 2 3600.00 +East Keyboard 2023 1 1800.00 +East Keyboard 2023 1 1800.00 +East Keyboard 2023 2 3600.00 +East Laptop \N 1 1800.00 +East Laptop \N 1 1800.00 +East Laptop \N 2 3600.00 +East Laptop 2023 1 1800.00 +East Laptop 2023 1 1800.00 +East Laptop 2023 2 3600.00 +North \N \N 2 3000.00 +North \N \N 3 3900.00 +North \N \N 5 6900.00 +North \N 2023 2 3000.00 +North \N 2023 3 3900.00 +North \N 2023 5 6900.00 +North Laptop \N 1 1500.00 +North Laptop \N 2 2400.00 +North Laptop \N 3 3900.00 +North Laptop 2023 1 1500.00 +North Laptop 2023 2 2400.00 +North Laptop 2023 3 3900.00 +North Mouse \N 1 1500.00 +North Mouse \N 1 1500.00 +North Mouse \N 2 3000.00 +North Mouse 2023 1 1500.00 +North Mouse 2023 1 1500.00 +North Mouse 2023 2 3000.00 +South \N \N 2 5000.00 +South \N \N 3 6200.00 +South \N \N 5 11200.00 +South \N 2023 2 5000.00 +South \N 2023 3 6200.00 +South \N 2023 5 11200.00 +South Keyboard \N 1 2500.00 +South Keyboard \N 2 3700.00 +South Keyboard \N 3 6200.00 +South Keyboard 2023 1 2500.00 +South Keyboard 2023 2 3700.00 +South Keyboard 2023 3 6200.00 +South Mouse \N 1 2500.00 +South Mouse \N 1 2500.00 +South Mouse \N 2 5000.00 +South Mouse 2023 1 2500.00 +South Mouse 2023 1 2500.00 +South Mouse 2023 2 5000.00 +West \N \N 1 800.00 +West \N \N 1 800.00 +West \N 2023 1 800.00 +West \N 2023 1 800.00 +West Mouse \N 1 800.00 +West Mouse \N 1 800.00 +West Mouse 2023 1 800.00 +West Mouse 2023 1 800.00 + +-- !sql_select -- +1 Laptop Electronics 5999.99 +1 Laptop High-End 5999.99 +1 Laptop Office 5999.99 +2 Mechanical Keyboard Accessories 399.99 +2 Mechanical Keyboard Electronics 399.99 +3 Basketball Outdoor 199.99 +3 Basketball Sports 199.99 +4 Badminton Racket Equipment 299.99 +4 Badminton Racket Sports 299.99 +5 Shirt Clothing 259.00 +5 Shirt Office 259.00 + +-- !sql_from -- +1 Laptop 0 Electronics +1 Laptop 1 Office +1 Laptop 2 High-End +2 Mechanical Keyboard 0 Electronics +2 Mechanical Keyboard 1 Accessories +3 Basketball 0 Sports +3 Basketball 1 Outdoor +4 Badminton Racket 0 Sports +4 Badminton Racket 1 Equipment +5 Shirt 0 Clothing +5 Shirt 1 Office + +-- !sql_cross_join -- +1 Laptop Digital Products +1 Laptop Office Supplies +2 Mechanical Keyboard Digital Products +3 Basketball Sports Equipment +4 Badminton Racket Sports Equipment +5 Shirt Apparel + +-- !sql_left_join -- +1 [80, 85, 87] 80 +1 [80, 85, 87] 85 +1 [80, 85, 87] 87 +2 [77, null, 89] \N +2 [77, null, 89] 77 +2 [77, null, 89] 89 +3 \N \N +4 [] \N +5 [90, 92] 90 +5 [90, 92] 92 + +-- !sql_group_by -- +Accessories 1 399.9900 +Clothing 1 259.0000 +Electronics 2 3199.9900 +Equipment 1 299.9900 +High-End 1 5999.9900 +Office 2 3129.4950 +Outdoor 1 199.9900 +Sports 2 249.9900 + +-- !sql_order_by -- +1 Laptop Electronics 5999.99 +1 Laptop High-End 5999.99 +1 Laptop Office 5999.99 +2 Mechanical Keyboard Accessories 399.99 +2 Mechanical Keyboard Electronics 399.99 +3 Basketball Outdoor 199.99 +3 Basketball Sports 199.99 +4 Badminton Racket Equipment 299.99 +4 Badminton Racket Sports 299.99 +5 Shirt Clothing 259.00 +5 Shirt Office 259.00 + +-- !sql_window -- +1 Laptop Electronics 5999.99 1 +1 Laptop High-End 5999.99 1 +1 Laptop Office 5999.99 1 +2 Mechanical Keyboard Accessories 399.99 1 +2 Mechanical Keyboard Electronics 399.99 2 +3 Basketball Outdoor 199.99 1 +3 Basketball Sports 199.99 2 +4 Badminton Racket Equipment 299.99 1 +4 Badminton Racket Sports 299.99 1 +5 Shirt Clothing 259.00 1 +5 Shirt Office 259.00 2 + +-- !sql_distinct -- +Accessories +Clothing +Electronics +Equipment +High-End +Office +Outdoor +Sports + +-- !sql_count -- +Badminton Racket 2 +Basketball 2 +Mechanical Keyboard 2 +Shirt 2 + +-- !sql_sum -- +Badminton Racket 3 +Basketball 3 +Laptop 3 + +-- !sql_count_window -- +Badminton Racket ["Sports", "Equipment"] Equipment 2 +Badminton Racket ["Sports", "Equipment"] Sports 2 +Basketball ["Sports", "Outdoor"] Outdoor 2 +Basketball ["Sports", "Outdoor"] Sports 2 +Laptop ["Electronics", "Office", "High-End"] Electronics 3 +Laptop ["Electronics", "Office", "High-End"] High-End 3 +Laptop ["Electronics", "Office", "High-End"] Office 3 +Mechanical Keyboard ["Electronics", "Accessories"] Accessories 2 +Mechanical Keyboard ["Electronics", "Accessories"] Electronics 2 +Shirt ["Clothing", "Office"] Clothing 2 +Shirt ["Clothing", "Office"] Office 2 + +-- !sql_sum_window -- +Badminton Racket [3] 3 6 +Basketball [3] 3 6 +Laptop [1, 2] 1 3 +Laptop [1, 2] 2 3 +Mechanical Keyboard [1] 1 1 +Shirt [4] 4 4 + diff --git a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy new file mode 100644 index 00000000000000..0c67e872af0dee --- /dev/null +++ b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy @@ -0,0 +1,365 @@ +// 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. + +suite("nereids_unnest_fn") { + sql 'use regression_test_nereids_function_p0' + + order_qt_sql_unnest_bitmap ''' + select id, e from fn_test, unnest(to_bitmap(kbint)) as tmp(e) order by id, e''' + order_qt_sql_unnest_bitmap_notnull ''' + select id, e from fn_test_not_nullable, unnest(to_bitmap(kbint)) as tmp(e) order by id, e''' + + order_qt_sql_unnest_outer_bitmap ''' + select id, e from fn_test left outer join unnest(to_bitmap(kbint)) as tmp(e) on true order by id, e''' + order_qt_sql_unnest_outer_bitmap_notnull ''' + select id, e from fn_test_not_nullable left outer join unnest(to_bitmap(kbint)) as tmp(e) on true order by id, e''' + + order_qt_sql_unnest_array "select id, e from fn_test, unnest(kaint) as tmp(e) order by id, e" + order_qt_sql_unnest_array_notnull "select id, e from fn_test_not_nullable, unnest(kaint) as tmp(e) order by id, e" + order_qt_sql_unnest_outer_array ''' + select id, e from fn_test left outer join unnest(kaint) as tmp(e) on true order by id, e''' + order_qt_sql_unnest_outer_array_notnull ''' + select id, e from fn_test_not_nullable left outer join unnest(kaint) as tmp(e) on true order by id, e''' + + order_qt_sql_unnest_map "select id, e, x from fn_test, unnest(km_int_int) as tmp(e, x) order by id, e" + order_qt_sql_unnest_map_notnull "select id, e, x from fn_test_not_nullable, unnest(km_int_int) as tmp(e, x) order by id, e" + order_qt_sql_unnest_outer_map ''' + select id, e, x from fn_test left outer join unnest(km_int_int) as tmp(e, x) on true order by id, e''' + order_qt_sql_unnest_outer_map_notnull ''' + select id, e, x from fn_test_not_nullable left outer join unnest(km_int_int) as tmp(e, x) on true order by id, e''' + + order_qt_sql_unnest_literal_from_1 '''SELECT * FROM unnest([1,2,3]);''' + order_qt_sql_unnest_literal_from_2 '''SELECT * FROM unnest([1,2,NULL,3], ['11',NULL,'22']);''' + order_qt_sql_unnest_literal_from_nested '''SELECT t.* FROM unnest([[1,2],[3],NULL,[4,5,6]]) AS t;''' + order_qt_sql_unnest_literal_from_hybrid '''SELECT t.* FROM unnest([[1,2],[3],NULL,[4,5,6]], ['hi','hello']) AS t(c1,c2);''' + order_qt_sql_unnest_literal_from_multiple '''SELECT * FROM unnest([1,2,3]) t1(c1), unnest(['11','22']) AS t2(c2);''' + + order_qt_sql_unnest_literal_select_1 '''SELECT unnest([1,2,3]);''' + order_qt_sql_unnest_literal_select_nested '''SELECT unnest([[1,2],[3],NULL,[4,5,6]]) AS t;''' + order_qt_sql_unnest_literal_select_multiple '''SELECT unnest([1,2,3]) AS c1, unnest(['11','22']) AS c2;''' + + multi_sql ''' + DROP TABLE IF EXISTS sales_unnest_t; + DROP TABLE IF EXISTS products_dict_unnest_t; + DROP TABLE IF EXISTS items_dict_unnest_t; + DROP TABLE IF EXISTS categories_dict_unnest_t; + DROP TABLE IF EXISTS student_unnest_t; + + CREATE TABLE sales_unnest_t ( + sale_id INT, + region VARCHAR(50), + products ARRAY, + amount DECIMAL(10, 2), + sale_date DATE + ) ENGINE=OLAP + DUPLICATE KEY(sale_id) + DISTRIBUTED BY HASH(sale_id) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE products_dict_unnest_t ( + product_id INT, + product_name VARCHAR(50) + ) ENGINE=OLAP + DUPLICATE KEY(product_id) + DISTRIBUTED BY HASH(product_id) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE items_dict_unnest_t ( + id INT, + name VARCHAR(50), + tags ARRAY, + price DECIMAL(10,2), + category_ids ARRAY + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE categories_dict_unnest_t ( + id INT KEY, + name VARCHAR(30) + ) ENGINE=OLAP + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE student_unnest_t + ( + `id` bigint(20) NULL COMMENT "", + `scores` ARRAY NULL COMMENT "" + ) + DUPLICATE KEY (id) + DISTRIBUTED BY HASH(`id`) + PROPERTIES ( + "replication_num" = "1" + ); + + INSERT INTO items_dict_unnest_t (id, name, tags, price, category_ids) VALUES + (1, 'Laptop', ['Electronics', 'Office', 'High-End'], 5999.99, [1, 2]), + (2, 'Mechanical Keyboard', ['Electronics', 'Accessories'], 399.99, [1]), + (3, 'Basketball', ['Sports', 'Outdoor'], 199.99, [3]), + (4, 'Badminton Racket', ['Sports', 'Equipment'], 299.99, [3]), + (5, 'Shirt', ['Clothing', 'Office'], 259.00, [4]); + + INSERT INTO categories_dict_unnest_t (id, name) VALUES + (1, 'Digital Products'), + (2, 'Office Supplies'), + (3, 'Sports Equipment'), + (4, 'Apparel'); + + INSERT INTO sales_unnest_t (sale_id, region, products, amount, sale_date) VALUES + (1, 'North', [101, 102], 1500.00, '2023-01-15'), + (2, 'North', [101], 900.00, '2023-01-20'), + (3, 'South', [102, 103], 2500.00, '2023-01-15'), + (4, 'South', [103], 1200.00, '2023-01-25'), + (5, 'East', [101, 103], 1800.00, '2023-01-18'), + (6, 'West', [102], 800.00, '2023-01-22'); + + INSERT INTO products_dict_unnest_t (product_id, product_name) VALUES + (101, 'Laptop'), + (102, 'Mouse'), + (103, 'Keyboard'); + + INSERT INTO student_unnest_t VALUES + (1, [80,85,87]), + (2, [77, null, 89]), + (3, null), + (4, []), + (5, [90,92]); + ''' + + order_qt_sql_grouping_set ''' + SELECT + region, + p.product_name, + COUNT(s.sale_id) AS order_count, + SUM(s.amount) AS total_sales + FROM + sales_unnest_t s + CROSS JOIN unnest(s.products) AS p_id + LEFT JOIN products_dict_unnest_t p ON p_id = p.product_id + GROUP BY + GROUPING SETS ( + (unnest(s.products)), + (region), + (p.product_name), + (region, p.product_name) + ) + ORDER BY + region, p.product_name; + ''' + + order_qt_sql_cube ''' + SELECT + region, + p.product_name, + EXTRACT(YEAR FROM s.sale_date) AS sale_year, + COUNT(s.sale_id) AS order_count, + SUM(s.amount) AS total_sales + FROM + sales_unnest_t s + CROSS JOIN unnest(s.products) AS p_id + LEFT JOIN products_dict_unnest_t p ON p_id = p.product_id + GROUP BY + CUBE ( + (unnest(s.products)), + region, + p.product_name, + EXTRACT(YEAR FROM s.sale_date) + ) + ORDER BY + region, p.product_name, sale_year; + ''' + + order_qt_sql_select ''' + SELECT + id, + name, + unnest(tags) AS tag, + price + FROM + items_dict_unnest_t + ORDER BY + id; + ''' + + order_qt_sql_from ''' + SELECT + i.id, + i.name, + t.tag, + t.ord + FROM + items_dict_unnest_t i, + unnest(i.tags) with ordinality AS t(tag, ord) + ORDER BY + i.id, + i.name; + ''' + + order_qt_sql_cross_join ''' + SELECT + i.id, + i.name, + c.name AS category_name + FROM + items_dict_unnest_t i + CROSS JOIN unnest(i.category_ids) AS cid + INNER JOIN categories_dict_unnest_t c ON cid = c.id + ORDER BY + i.id, + i.name; + ''' + + order_qt_sql_left_join ''' + SELECT + id, + scores, + unnest + FROM + student_unnest_t + LEFT JOIN unnest(scores) AS unnest ON TRUE + ORDER BY + 1, + 3; + ''' + + order_qt_sql_group_by ''' + SELECT + unnest(i.tags), + COUNT(DISTINCT i.id) AS item_count, + AVG(i.price) AS avg_price + FROM + items_dict_unnest_t i + GROUP BY + unnest(i.tags) + ORDER BY + unnest(i.tags); + ''' + + order_qt_sql_order_by ''' + SELECT + id, + name, + unnest(tags), + price + FROM items_dict_unnest_t + ORDER BY + unnest(tags), + price DESC; + ''' + + order_qt_sql_window ''' + SELECT + id, + name, + unnest(tags) AS tag, + price, + RANK() OVER ( + PARTITION BY unnest(tags) + ORDER BY + price DESC + ) AS price_rank + FROM + items_dict_unnest_t + ORDER BY + id, + name; + ''' + + order_qt_sql_distinct ''' + SELECT + DISTINCT unnest(tags) AS unique_tag + FROM + items_dict_unnest_t + ORDER BY + unnest(tags); + ''' + + order_qt_sql_count ''' + select + name, + count(unnest(tags)) + from + items_dict_unnest_t + group by + name + having + count(unnest(tags)) = 2 + order by + name; + ''' + + order_qt_sql_sum ''' + select + name, + sum(unnest(category_ids)) + from + items_dict_unnest_t + group by + name + having + sum(unnest(category_ids)) = 3 + order by + name; + ''' + + order_qt_sql_count_window ''' + select + name, + tags, + unnest(tags), + count(unnest(tags)) over(partition by tags) + from + items_dict_unnest_t + group by + name, tags, unnest(tags) + order by + name, tags, unnest(tags); + ''' + + order_qt_sql_sum_window ''' + select + name, + category_ids, + unnest(category_ids), + sum(unnest(category_ids)) over(partition by category_ids) + from + items_dict_unnest_t + group by + name, category_ids, unnest(category_ids) + order by + name, category_ids, unnest(category_ids); + ''' + + // order_qt_sql_join_complex ''' + // SELECT + // id, + // tags + // FROM + // items_dict_unnest_t + // LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + // ''' +} From 88fdce19a7b436e5166d059bf9d6b06345daf3de Mon Sep 17 00:00:00 2001 From: lichi Date: Fri, 21 Nov 2025 17:03:22 +0800 Subject: [PATCH 02/14] fix failed ut --- .../nereids/trees/plans/logical/LogicalGenerate.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java index 1f7dc0c7b724df..b38830bdaf94d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java @@ -191,10 +191,12 @@ public String toDigest() { .append(" AS ") .append( expandColumnAlias.get(0).stream().collect(Collectors.joining(", ")) - ) - .append(" WHERE ") - .append(!conjuncts.isEmpty() ? conjuncts.stream().map(Expression::toDigest) - .collect(Collectors.joining(" AND ")) : "TRUE"); + ); + if (!conjuncts.isEmpty()) { + sb.append(" WHERE ") + .append(conjuncts.stream().map(Expression::toDigest).collect(Collectors.joining(" AND "))); + } + return sb.toString(); } From e14347af21754cbec2be00c28c0f0b3bf84ac2d5 Mon Sep 17 00:00:00 2001 From: lichi Date: Thu, 11 Dec 2025 15:26:52 +0800 Subject: [PATCH 03/14] rebase code --- .../java/org/apache/doris/nereids/util/ExpressionUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index a2d25337982dc6..5797f77af84cce 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -342,8 +342,8 @@ public static List shuttleExpressionWithLineage(List Date: Fri, 12 Dec 2025 17:14:03 +0800 Subject: [PATCH 04/14] update code --- .../rules/analysis/NormalizeAggregate.java | 46 +++++++++++-------- .../doris/nereids/util/ExpressionUtils.java | 4 +- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java index 25cf0cbaa1040b..fb31810def5aeb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregate.java @@ -163,8 +163,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional aggregateOutput = aggregate.getOutputExpressions(); - Map> aggFuncs = CollectNonWindowedAggFuncsWithSessionVar - .collect(aggregateOutput); + Map> aggFuncs = + CollectNonWindowedAggFuncsWithSessionVar.collect(aggregateOutput); // split agg child as two part // TRUE part 1: need push down itself, if it contains subquery or window expression @@ -194,8 +194,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional inputSlots = arg instanceof OrderExpression ? arg.getInputSlots() - : ImmutableList.of(arg); + Collection inputSlots + = arg instanceof OrderExpression ? arg.getInputSlots() : ImmutableList.of(arg); for (Expression input : inputSlots) { if (input instanceof SlotReference) { needPushDownInputs.add(input); @@ -210,7 +210,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional needPushSelf = needPushDownSelfExprs.build(); Set needPushInputSlots = ExpressionUtils.getInputSlotSet(needPushDownInputs.build()); - Set existsAlias = ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); + Set existsAlias = + ExpressionUtils.mutableCollect(aggregateOutput, Alias.class::isInstance); // push down 3 kinds of exprs, these pushed exprs will be used to normalize agg output later // 1. group by exprs @@ -224,10 +225,12 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional pushedGroupByExprs = bottomSlotContext.pushDownToNamedExpression(groupingByExprs); - Set pushedTrivialAggChildren = bottomSlotContext.pushDownToNamedExpression(needPushSelf); - Set pushedTrivialAggInputSlots = bottomSlotContext - .pushDownToNamedExpression(needPushInputSlots); + Set pushedGroupByExprs = + bottomSlotContext.pushDownToNamedExpression(groupingByExprs); + Set pushedTrivialAggChildren = + bottomSlotContext.pushDownToNamedExpression(needPushSelf); + Set pushedTrivialAggInputSlots = + bottomSlotContext.pushDownToNamedExpression(needPushInputSlots); Set bottomProjects = Sets.union(pushedGroupByExprs, Sets.union(pushedTrivialAggChildren, pushedTrivialAggInputSlots)); @@ -243,11 +246,12 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional normalizedGroupExprs = bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); + List normalizedGroupExprs = + bottomSlotContext.normalizeToUseSlotRef(groupingByExprs); // normalize trivial-aggs by bottomProjects - List normalizedAggFuncs = bottomSlotContext - .normalizeToUseSlotRef(SessionVarGuardExpr.getExprWithGuard(aggFuncs)); + List normalizedAggFuncs = + bottomSlotContext.normalizeToUseSlotRef(SessionVarGuardExpr.getExprWithGuard(aggFuncs)); if (normalizedAggFuncs.stream().anyMatch(agg -> !agg.children().isEmpty() && agg.child(0).containsType(AggregateFunction.class))) { throw new AnalysisException( @@ -255,19 +259,20 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional normalizedAggOutputBuilder = ImmutableList - .builderWithExpectedSize(groupingByExprs.size() + normalizedAggFuncs.size()); + ImmutableList.Builder normalizedAggOutputBuilder + = ImmutableList.builderWithExpectedSize(groupingByExprs.size() + normalizedAggFuncs.size()); for (NamedExpression pushedGroupByExpr : pushedGroupByExprs) { normalizedAggOutputBuilder.add(pushedGroupByExpr.toSlot()); } normalizedAggOutputBuilder.addAll( - normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs)); + normalizedAggFuncsToSlotContext.pushDownToNamedExpression(normalizedAggFuncs) + ); // create new agg node ImmutableList normalizedAggOutput = normalizedAggOutputBuilder.build(); @@ -321,8 +326,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional newAggregate = aggregate.withNormalized(normalizedGroupExprs, - normalizedAggOutputBuilder.build(), bottomPlan); + LogicalAggregate newAggregate = + aggregate.withNormalized(normalizedGroupExprs, normalizedAggOutputBuilder.build(), bottomPlan); ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(ctx); LogicalProject project = eliminateGroupByConstant(groupByExprContext, rewriteContext, normalizedGroupExprs, normalizedAggOutput, bottomProjects, aggregate, upperProjects, newAggregate); @@ -338,7 +343,8 @@ private LogicalPlan normalizeAgg(LogicalAggregate aggregate, Optional( ExpressionUtils.replace(having.get().getConjuncts(), project.getAliasToProducer()), - project.child()))); + project.child() + ))); } // after build logical plan, it will not extract window expression from the SELECT lists, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 5797f77af84cce..830fd4772a9828 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -1319,8 +1319,8 @@ public static org.apache.doris.nereids.trees.expressions.functions.Function conv if (dataType.isArrayType()) { Expression[] arrayArgs = args.toArray(new Expression[0]); return unnest.isOuter() - ? unnest.needOrdinality() ? new PosExplodeOuter(args.get(0)) : new ExplodeOuter(arrayArgs) - : unnest.needOrdinality() ? new PosExplode(args.get(0)) : new Explode(arrayArgs); + ? unnest.needOrdinality() ? new PosExplodeOuter(arrayArgs) : new ExplodeOuter(arrayArgs) + : unnest.needOrdinality() ? new PosExplode(arrayArgs) : new Explode(arrayArgs); } else { if (unnest.needOrdinality()) { throw new AnalysisException(String.format("only ARRAY support WITH ORDINALITY," From 9102a86e0e8d4b62861012c8b4dfc8a0f2c11cbd Mon Sep 17 00:00:00 2001 From: jacktengg Date: Mon, 15 Dec 2025 16:55:47 +0800 Subject: [PATCH 05/14] be support expland_conjuncts --- .../pipeline/exec/table_function_operator.cpp | 106 +++++++++++++++--- .../pipeline/exec/table_function_operator.h | 8 +- .../gen_function/unnest.out | 20 +++- .../gen_function/unnest.groovy | 20 ++-- 4 files changed, 125 insertions(+), 29 deletions(-) diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 7b99da67f0773b..1755ffdced666b 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -17,10 +17,13 @@ #include "table_function_operator.h" +#include #include #include "pipeline/exec/operator.h" +#include "util/simd/bits.h" #include "vec/core/block.h" +#include "vec/core/column_numbers.h" #include "vec/exprs/table_function/table_function_factory.h" namespace doris { @@ -55,6 +58,11 @@ Status TableFunctionLocalState::_clone_table_function(RuntimeState* state) { fn->set_expr_context(_vfn_ctxs[i]); _fns.push_back(fn); } + _expand_conjuncts_ctxs.resize(p._expand_conjuncts_ctxs.size()); + for (size_t i = 0; i < _expand_conjuncts_ctxs.size(); i++) { + RETURN_IF_ERROR(p._expand_conjuncts_ctxs[i]->clone(state, _expand_conjuncts_ctxs[i])); + } + _need_to_handle_outer_conjuncts = (!_expand_conjuncts_ctxs.empty() && _fns[0]->is_outer()); return Status::OK(); } @@ -70,12 +78,11 @@ Status TableFunctionLocalState::open(RuntimeState* state) { return Status::OK(); } -void TableFunctionLocalState::_copy_output_slots( - std::vector& columns) { +void TableFunctionLocalState::_copy_output_slots(std::vector& columns, + const TableFunctionOperatorX& p) { if (!_current_row_insert_times) { return; } - auto& p = _parent->cast(); for (auto index : p._output_slot_indexs) { auto src_column = _child_block->get_by_position(index).column; columns[index]->insert_many_from(*src_column, _cur_child_offset, _current_row_insert_times); @@ -159,25 +166,38 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, output_block, p._output_slots); vectorized::MutableColumns& columns = m_block.mutable_columns(); + auto child_slot_count = p._child_slots.size(); for (int i = 0; i < p._fn_num; i++) { - if (columns[i + p._child_slots.size()]->is_nullable()) { + if (columns[i + child_slot_count]->is_nullable()) { _fns[i]->set_nullable(); } } + std::vector child_row_to_output_rows_indices; + std::vector handled_row_indices; + bool child_block_empty = _child_block->empty(); + if (_need_to_handle_outer_conjuncts && !child_block_empty) { + child_row_to_output_rows_indices.push_back(0); + } SCOPED_TIMER(_process_rows_timer); - while (columns[p._child_slots.size()]->size() < state->batch_size()) { + auto batch_size = state->batch_size(); + int32_t max_batch_size = std::numeric_limits::max(); + while (columns[child_slot_count]->size() < batch_size) { RETURN_IF_CANCELLED(state); - if (_child_block->rows() == 0) { + if (_child_block->rows() == 0 || _cur_child_offset == -1) { break; } bool skip_child_row = false; - while (columns[p._child_slots.size()]->size() < state->batch_size()) { + while (columns[child_slot_count]->size() < batch_size) { int idx = _find_last_fn_eos_idx(); if (idx == 0 || skip_child_row) { - _copy_output_slots(columns); + _copy_output_slots(columns, p); // all table functions' results are exhausted, process next child row. + if (_need_to_handle_outer_conjuncts) { + handled_row_indices.push_back(_cur_child_offset); + child_row_to_output_rows_indices.push_back(columns[child_slot_count]->size()); + } process_next_child_row(); if (_cur_child_offset == -1) { break; @@ -197,24 +217,62 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, DCHECK_LE(1, p._fn_num); auto repeat_times = _fns[p._fn_num - 1]->get_value( - columns[p._child_slots.size() + p._fn_num - 1], + columns[child_slot_count + p._fn_num - 1], //// It has already been checked that // columns[p._child_slots.size()]->size() < state->batch_size(), // so columns[p._child_slots.size()]->size() will not exceed the range of int. - state->batch_size() - (int)columns[p._child_slots.size()]->size()); + _need_to_handle_outer_conjuncts + ? max_batch_size + : batch_size - (int)columns[child_slot_count]->size()); _current_row_insert_times += repeat_times; for (int i = 0; i < p._fn_num - 1; i++) { - _fns[i]->get_same_many_values(columns[i + p._child_slots.size()], repeat_times); + _fns[i]->get_same_many_values(columns[i + child_slot_count], repeat_times); } } } - _copy_output_slots(columns); + _copy_output_slots(columns, p); - size_t row_size = columns[p._child_slots.size()]->size(); + size_t row_size = columns[child_slot_count]->size(); for (auto index : p._useless_slot_indexs) { columns[index]->insert_many_defaults(row_size - columns[index]->size()); } + if (!_expand_conjuncts_ctxs.empty() && !child_block_empty) { + auto output_row_count = output_block->rows(); + vectorized::IColumn::Filter filter; + auto column_count = output_block->columns(); + vectorized::ColumnNumbers columns_to_filter(column_count); + std::iota(columns_to_filter.begin(), columns_to_filter.end(), 0); + RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts_and_filter_block( + _expand_conjuncts_ctxs, output_block, columns_to_filter, column_count, filter)); + size_t remain_row_count = output_block->rows(); + // for outer table function, need to handle those child rows which all expanded rows are filtered out + if (_need_to_handle_outer_conjuncts && remain_row_count < output_row_count) { + auto handled_child_row_count = handled_row_indices.size(); + for (size_t i = 0; i < handled_child_row_count; ++i) { + auto start_row_idx = child_row_to_output_rows_indices[i]; + auto end_row_idx = child_row_to_output_rows_indices[i + 1]; + if (!simd::contain_byte((uint8_t*)filter.data() + start_row_idx, + end_row_idx - start_row_idx, 1)) { + for (auto index : p._output_slot_indexs) { + auto src_column = _child_block->get_by_position(index).column; + columns[index]->insert_from(*src_column, handled_row_indices[i]); + } + for (auto index : p._useless_slot_indexs) { + columns[index]->insert_default(); + } + for (int j = 0; j != p._fn_num; j++) { + columns[j + child_slot_count]->insert_default(); + } + } + } + } + if (_cur_child_offset == -1) { + _child_block->clear_column_data(_parent->cast() + ._child->row_desc() + .num_materialized_slots()); + } + } { SCOPED_TIMER(_filter_timer); // 3. eval conjuncts @@ -235,9 +293,13 @@ void TableFunctionLocalState::process_next_child_row() { fn->process_close(); } - _child_block->clear_column_data(_parent->cast() - ._child->row_desc() - .num_materialized_slots()); + // if there are any _expand_conjuncts_ctxs, don't clear child block here, + // because we still need _child_block to output NULL rows for outer table function + if (_expand_conjuncts_ctxs.empty()) { + _child_block->clear_column_data(_parent->cast() + ._child->row_desc() + .num_materialized_slots()); + } _cur_child_offset = -1; return; } @@ -284,6 +346,15 @@ Status TableFunctionOperatorX::init(const TPlanNode& tnode, RuntimeState* state) } _fn_num = cast_set(_fns.size()); + for (const TExpr& texpr : tnode.table_function_node.expand_conjuncts) { + vectorized::VExprContextSPtr ctx; + RETURN_IF_ERROR(vectorized::VExpr::create_expr_tree(texpr, ctx)); + _expand_conjuncts_ctxs.push_back(ctx); + } + if (!_expand_conjuncts_ctxs.empty()) { + DCHECK(1 == _fn_num) << "Only support one table function when there are expand conjuncts."; + } + // Prepare output slot ids RETURN_IF_ERROR(_prepare_output_slot_ids(tnode)); return Status::OK(); @@ -296,6 +367,8 @@ Status TableFunctionOperatorX::prepare(doris::RuntimeState* state) { } RETURN_IF_ERROR(vectorized::VExpr::prepare(_vfn_ctxs, state, row_descriptor())); + RETURN_IF_ERROR(vectorized::VExpr::prepare(_expand_conjuncts_ctxs, state, row_descriptor())); + // get current all output slots for (const auto& tuple_desc : row_descriptor().tuple_descriptors()) { for (const auto& slot_desc : tuple_desc->slots()) { @@ -318,6 +391,7 @@ Status TableFunctionOperatorX::prepare(doris::RuntimeState* state) { } } + RETURN_IF_ERROR(vectorized::VExpr::open(_expand_conjuncts_ctxs, state)); return vectorized::VExpr::open(_vfn_ctxs, state); } diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 9d5dd6e820a150..c5446f34db0dc6 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -19,6 +19,8 @@ #include +#include + #include "common/status.h" #include "operator.h" #include "vec/exprs/table_function/table_function.h" @@ -56,7 +58,8 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> MOCK_FUNCTION Status _clone_table_function(RuntimeState* state); - void _copy_output_slots(std::vector& columns); + void _copy_output_slots(std::vector& columns, + const TableFunctionOperatorX& p); bool _roll_table_functions(int last_eos_idx); // return: // 0: all fns are eos @@ -67,6 +70,8 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> std::vector _fns; vectorized::VExprContextSPtrs _vfn_ctxs; + vectorized::VExprContextSPtrs _expand_conjuncts_ctxs; + bool _need_to_handle_outer_conjuncts = false; int64_t _cur_child_offset = -1; std::unique_ptr _child_block; int _current_row_insert_times = 0; @@ -150,6 +155,7 @@ class TableFunctionOperatorX MOCK_REMOVE(final) std::vector _output_slots; vectorized::VExprContextSPtrs _vfn_ctxs; + vectorized::VExprContextSPtrs _expand_conjuncts_ctxs; std::vector _fns; int _fn_num = 0; diff --git a/regression-test/data/nereids_function_p0/gen_function/unnest.out b/regression-test/data/nereids_function_p0/gen_function/unnest.out index 5553db4a89bb2f..a6f41b8a608887 100644 --- a/regression-test/data/nereids_function_p0/gen_function/unnest.out +++ b/regression-test/data/nereids_function_p0/gen_function/unnest.out @@ -431,6 +431,7 @@ West Mouse 2023 1 800.00 -- !sql_select -- 1 Laptop Electronics 5999.99 1 Laptop High-End 5999.99 +1 Laptop Laptop 5999.99 1 Laptop Office 5999.99 2 Mechanical Keyboard Accessories 399.99 2 Mechanical Keyboard Electronics 399.99 @@ -445,6 +446,7 @@ West Mouse 2023 1 800.00 1 Laptop 0 Electronics 1 Laptop 1 Office 1 Laptop 2 High-End +1 Laptop 3 Laptop 2 Mechanical Keyboard 0 Electronics 2 Mechanical Keyboard 1 Accessories 3 Basketball 0 Sports @@ -480,6 +482,7 @@ Clothing 1 259.0000 Electronics 2 3199.9900 Equipment 1 299.9900 High-End 1 5999.9900 +Laptop 1 5999.9900 Office 2 3129.4950 Outdoor 1 199.9900 Sports 2 249.9900 @@ -487,6 +490,7 @@ Sports 2 249.9900 -- !sql_order_by -- 1 Laptop Electronics 5999.99 1 Laptop High-End 5999.99 +1 Laptop Laptop 5999.99 1 Laptop Office 5999.99 2 Mechanical Keyboard Accessories 399.99 2 Mechanical Keyboard Electronics 399.99 @@ -500,6 +504,7 @@ Sports 2 249.9900 -- !sql_window -- 1 Laptop Electronics 5999.99 1 1 Laptop High-End 5999.99 1 +1 Laptop Laptop 5999.99 1 1 Laptop Office 5999.99 1 2 Mechanical Keyboard Accessories 399.99 1 2 Mechanical Keyboard Electronics 399.99 2 @@ -516,6 +521,7 @@ Clothing Electronics Equipment High-End +Laptop Office Outdoor Sports @@ -536,9 +542,10 @@ Badminton Racket ["Sports", "Equipment"] Equipment 2 Badminton Racket ["Sports", "Equipment"] Sports 2 Basketball ["Sports", "Outdoor"] Outdoor 2 Basketball ["Sports", "Outdoor"] Sports 2 -Laptop ["Electronics", "Office", "High-End"] Electronics 3 -Laptop ["Electronics", "Office", "High-End"] High-End 3 -Laptop ["Electronics", "Office", "High-End"] Office 3 +Laptop ["Electronics", "Office", "High-End", "Laptop"] Electronics 4 +Laptop ["Electronics", "Office", "High-End", "Laptop"] High-End 4 +Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop 4 +Laptop ["Electronics", "Office", "High-End", "Laptop"] Office 4 Mechanical Keyboard ["Electronics", "Accessories"] Accessories 2 Mechanical Keyboard ["Electronics", "Accessories"] Electronics 2 Shirt ["Clothing", "Office"] Clothing 2 @@ -552,3 +559,10 @@ Laptop [1, 2] 2 3 Mechanical Keyboard [1] 1 1 Shirt [4] 4 4 +-- !sql_join_complex -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop +2 Mechanical Keyboard ["Electronics", "Accessories"] \N +3 Basketball ["Sports", "Outdoor"] \N +4 Badminton Racket ["Sports", "Equipment"] \N +5 Shirt ["Clothing", "Office"] \N + diff --git a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy index 0c67e872af0dee..8efc120ab9a612 100644 --- a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy +++ b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy @@ -117,7 +117,7 @@ suite("nereids_unnest_fn") { ); INSERT INTO items_dict_unnest_t (id, name, tags, price, category_ids) VALUES - (1, 'Laptop', ['Electronics', 'Office', 'High-End'], 5999.99, [1, 2]), + (1, 'Laptop', ['Electronics', 'Office', 'High-End', 'Laptop'], 5999.99, [1, 2]), (2, 'Mechanical Keyboard', ['Electronics', 'Accessories'], 399.99, [1]), (3, 'Basketball', ['Sports', 'Outdoor'], 199.99, [3]), (4, 'Badminton Racket', ['Sports', 'Equipment'], 299.99, [3]), @@ -354,12 +354,14 @@ suite("nereids_unnest_fn") { name, category_ids, unnest(category_ids); ''' - // order_qt_sql_join_complex ''' - // SELECT - // id, - // tags - // FROM - // items_dict_unnest_t - // LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; - // ''' + order_qt_sql_join_complex ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' } From c5a67942316d271b0245191ae62d84e454166e91 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Tue, 16 Dec 2025 10:54:34 +0800 Subject: [PATCH 06/14] improve and add tests --- .../pipeline/exec/table_function_operator.cpp | 114 +++++++++++++-- .../pipeline/exec/table_function_operator.h | 7 +- .../operator/table_function_operator_test.cpp | 1 - .../gen_function/unnest.out | 124 ++++++++++++++-- .../gen_function/unnest.groovy | 132 +++++++++++++++++- 5 files changed, 347 insertions(+), 31 deletions(-) diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 1755ffdced666b..0a53d858d5c93d 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -78,8 +78,10 @@ Status TableFunctionLocalState::open(RuntimeState* state) { return Status::OK(); } -void TableFunctionLocalState::_copy_output_slots(std::vector& columns, - const TableFunctionOperatorX& p) { +void TableFunctionLocalState::_copy_output_slots( + std::vector& columns, const TableFunctionOperatorX& p, + size_t output_row_count, std::vector& child_row_to_output_rows_indices, + std::vector& handled_row_indices) { if (!_current_row_insert_times) { return; } @@ -88,6 +90,11 @@ void TableFunctionLocalState::_copy_output_slots(std::vectorinsert_many_from(*src_column, _cur_child_offset, _current_row_insert_times); } _current_row_insert_times = 0; + + if (_need_to_handle_outer_conjuncts) { + handled_row_indices.push_back(_cur_child_offset); + child_row_to_output_rows_indices.push_back(output_row_count); + } } // Returns the index of fn of the last eos counted from back to front @@ -165,7 +172,6 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, vectorized::MutableBlock m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block( output_block, p._output_slots); vectorized::MutableColumns& columns = m_block.mutable_columns(); - auto child_slot_count = p._child_slots.size(); for (int i = 0; i < p._fn_num; i++) { if (columns[i + child_slot_count]->is_nullable()) { @@ -184,20 +190,19 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, while (columns[child_slot_count]->size() < batch_size) { RETURN_IF_CANCELLED(state); + // finished handling current child block if (_child_block->rows() == 0 || _cur_child_offset == -1) { break; } bool skip_child_row = false; - while (columns[child_slot_count]->size() < batch_size) { + size_t cur_row_count = 0; + while (cur_row_count = columns[child_slot_count]->size(), cur_row_count < batch_size) { int idx = _find_last_fn_eos_idx(); if (idx == 0 || skip_child_row) { - _copy_output_slots(columns, p); + _copy_output_slots(columns, p, cur_row_count, child_row_to_output_rows_indices, + handled_row_indices); // all table functions' results are exhausted, process next child row. - if (_need_to_handle_outer_conjuncts) { - handled_row_indices.push_back(_cur_child_offset); - child_row_to_output_rows_indices.push_back(columns[child_slot_count]->size()); - } process_next_child_row(); if (_cur_child_offset == -1) { break; @@ -216,6 +221,11 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, } DCHECK_LE(1, p._fn_num); + // It may take multiple iterations of this while loop to process a child row if + // any table function produces a large number of rows. + // But for table function with outer conjuncts, we output all rows in one iteration + // even if it may exceed batch_size, because we need to make sure at least one row + // is output for the child row after evaluating the outer conjuncts. auto repeat_times = _fns[p._fn_num - 1]->get_value( columns[child_slot_count + p._fn_num - 1], //// It has already been checked that @@ -226,17 +236,84 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, : batch_size - (int)columns[child_slot_count]->size()); _current_row_insert_times += repeat_times; for (int i = 0; i < p._fn_num - 1; i++) { + LOG(INFO) << "xxxx Query: " << print_id(state->query_id()) + << " get_same_many_values for fn index: " << i; _fns[i]->get_same_many_values(columns[i + child_slot_count], repeat_times); } } } - _copy_output_slots(columns, p); + _copy_output_slots(columns, p, columns[child_slot_count]->size(), + child_row_to_output_rows_indices, handled_row_indices); size_t row_size = columns[child_slot_count]->size(); for (auto index : p._useless_slot_indexs) { columns[index]->insert_many_defaults(row_size - columns[index]->size()); } + // LOG(INFO) << "xxxx before filter output_block data: " + // << output_block->dump_data(0, output_block->rows()); + + /** + Handle the outer conjuncts after unnest. Currently, only left outer is supported. + e.g., for the following example data, + select id, name, tags from items_dict_unnest_t order by id; + +------+---------------------+-------------------------------------------------+ + | id | name | tags | + +------+---------------------+-------------------------------------------------+ + | 1 | Laptop | ["Electronics", "Office", "High-End", "Laptop"] | + | 2 | Mechanical Keyboard | ["Electronics", "Accessories"] | + | 3 | Basketball | ["Sports", "Outdoor"] | + | 4 | Badminton Racket | ["Sports", "Equipment"] | + | 5 | Shirt | ["Clothing", "Office", "Shirt"] | + +------+---------------------+-------------------------------------------------+ + + for this query: ``` SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name;``` + + after unnest, before evaluating the outer conjuncts, the result is: + +------+---------------------+-------------------------------------------------+--------------+ + | id | name | tags | unnest(tags) | + +------+---------------------+-------------------------------------------------+--------------+ + | 1 | Laptop | ["Electronics", "Office", "High-End", "Laptop"] | Electronics | + | 1 | Laptop | ["Electronics", "Office", "High-End", "Laptop"] | Office | + | 1 | Laptop | ["Electronics", "Office", "High-End", "Laptop"] | High-End | + | 1 | Laptop | ["Electronics", "Office", "High-End", "Laptop"] | Laptop | + | 2 | Mechanical Keyboard | ["Electronics", "Accessories"] | Electronics | + | 2 | Mechanical Keyboard | ["Electronics", "Accessories"] | Accessories | + | 3 | Basketball | ["Sports", "Outdoor"] | Sports | + | 3 | Basketball | ["Sports", "Outdoor"] | Outdoor | + | 4 | Badminton Racket | ["Sports", "Equipment"] | Sports | + | 4 | Badminton Racket | ["Sports", "Equipment"] | Equipment | + | 5 | Shirt | ["Clothing", "Office", "Shirt"] | Clothing | + | 5 | Shirt | ["Clothing", "Office", "Shirt"] | Office | + | 5 | Shirt | ["Clothing", "Office", "Shirt"] | Shirt | + +------+---------------------+-------------------------------------------------+--------------+ + 13 rows in set (0.47 sec) + + the vector child_row_to_output_rows_indices is used to record the mapping relationship, + between child row and output rows, for example: + child row 0 -> output rows [0, 4) + child row 1 -> output rows [4, 6) + child row 2 -> output rows [6, 8) + child row 3 -> output rows [8, 10) + child row 4 -> output rows [10, 13) + it's contents are: [0, 4, 6, 8, 10, 13]. + + After evaluating the left join conjuncts `t.tag = name`, + the content of filter is: [0, 0, 0, 1, // child row 0 + 0, 0, // child row 1 + 0, 0, // child row 2 + 0, 0, // child row 3 + 0, 0, 1 // child row 4 + ] + child rows 1, 2, 3 are all filtered out, so we need to insert one row with NULL tag value for each of them. + */ if (!_expand_conjuncts_ctxs.empty() && !child_block_empty) { auto output_row_count = output_block->rows(); vectorized::IColumn::Filter filter; @@ -246,6 +323,22 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts_and_filter_block( _expand_conjuncts_ctxs, output_block, columns_to_filter, column_count, filter)); size_t remain_row_count = output_block->rows(); + /* + LOG(INFO) << "xxxx after filter output_block row count: " << remain_row_count; + std::string debug_string("xxxx handled child rows: "); + for (auto v : handled_row_indices) { + debug_string += std::to_string(v) + ","; + } + debug_string += ", filter: "; + for (auto v : filter) { + debug_string += std::to_string(v) + ","; + } + debug_string += ", child_row_to_output_rows_indices: "; + for (auto v : child_row_to_output_rows_indices) { + debug_string += ", " + std::to_string(v); + } + LOG(INFO) << debug_string; + */ // for outer table function, need to handle those child rows which all expanded rows are filtered out if (_need_to_handle_outer_conjuncts && remain_row_count < output_row_count) { auto handled_child_row_count = handled_row_indices.size(); @@ -281,6 +374,7 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, } *eos = _child_eos && _cur_child_offset == -1; + // LOG(INFO) << "xxxx output_block rows: " << output_block->rows(); return Status::OK(); } diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index c5446f34db0dc6..61f48469a68f97 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -19,8 +19,6 @@ #include -#include - #include "common/status.h" #include "operator.h" #include "vec/exprs/table_function/table_function.h" @@ -59,7 +57,9 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> MOCK_FUNCTION Status _clone_table_function(RuntimeState* state); void _copy_output_slots(std::vector& columns, - const TableFunctionOperatorX& p); + const TableFunctionOperatorX& p, size_t output_row_count, + std::vector& child_row_to_output_rows_indices, + std::vector& handled_row_indices); bool _roll_table_functions(int last_eos_idx); // return: // 0: all fns are eos @@ -71,6 +71,7 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> std::vector _fns; vectorized::VExprContextSPtrs _vfn_ctxs; vectorized::VExprContextSPtrs _expand_conjuncts_ctxs; + // for table function with outer conjuncts, need to handle those child rows which all expanded rows are filtered out bool _need_to_handle_outer_conjuncts = false; int64_t _cur_child_offset = -1; std::unique_ptr _child_block; diff --git a/be/test/pipeline/operator/table_function_operator_test.cpp b/be/test/pipeline/operator/table_function_operator_test.cpp index ef8f1e23f9c870..ee555fa4efe6f4 100644 --- a/be/test/pipeline/operator/table_function_operator_test.cpp +++ b/be/test/pipeline/operator/table_function_operator_test.cpp @@ -156,7 +156,6 @@ TEST_F(TableFunctionOperatorTest, single_fn_test) { } { - Block block = ColumnHelper::create_block({1}); *local_state->_child_block = ColumnHelper::create_block({1}); auto st = op->push(state.get(), local_state->_child_block.get(), true); EXPECT_TRUE(st) << st.msg(); diff --git a/regression-test/data/nereids_function_p0/gen_function/unnest.out b/regression-test/data/nereids_function_p0/gen_function/unnest.out index a6f41b8a608887..268d7984e6d0f6 100644 --- a/regression-test/data/nereids_function_p0/gen_function/unnest.out +++ b/regression-test/data/nereids_function_p0/gen_function/unnest.out @@ -441,6 +441,7 @@ West Mouse 2023 1 800.00 4 Badminton Racket Sports 299.99 5 Shirt Clothing 259.00 5 Shirt Office 259.00 +5 Shirt Shirt 259.00 -- !sql_from -- 1 Laptop 0 Electronics @@ -455,11 +456,15 @@ West Mouse 2023 1 800.00 4 Badminton Racket 1 Equipment 5 Shirt 0 Clothing 5 Shirt 1 Office +5 Shirt 2 Shirt -- !sql_cross_join -- 1 Laptop Digital Products 1 Laptop Office Supplies +1 Laptop Sports Equipment 2 Mechanical Keyboard Digital Products +2 Mechanical Keyboard Office Supplies +3 Basketball Digital Products 3 Basketball Sports Equipment 4 Badminton Racket Sports Equipment 5 Shirt Apparel @@ -485,6 +490,7 @@ High-End 1 5999.9900 Laptop 1 5999.9900 Office 2 3129.4950 Outdoor 1 199.9900 +Shirt 1 259.0000 Sports 2 249.9900 -- !sql_order_by -- @@ -500,6 +506,7 @@ Sports 2 249.9900 4 Badminton Racket Sports 299.99 5 Shirt Clothing 259.00 5 Shirt Office 259.00 +5 Shirt Shirt 259.00 -- !sql_window -- 1 Laptop Electronics 5999.99 1 @@ -514,6 +521,7 @@ Sports 2 249.9900 4 Badminton Racket Sports 299.99 1 5 Shirt Clothing 259.00 1 5 Shirt Office 259.00 2 +5 Shirt Shirt 259.00 1 -- !sql_distinct -- Accessories @@ -524,18 +532,17 @@ High-End Laptop Office Outdoor +Shirt Sports -- !sql_count -- Badminton Racket 2 Basketball 2 Mechanical Keyboard 2 -Shirt 2 -- !sql_sum -- Badminton Racket 3 -Basketball 3 -Laptop 3 +Mechanical Keyboard 3 -- !sql_count_window -- Badminton Racket ["Sports", "Equipment"] Equipment 2 @@ -548,21 +555,114 @@ Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop 4 Laptop ["Electronics", "Office", "High-End", "Laptop"] Office 4 Mechanical Keyboard ["Electronics", "Accessories"] Accessories 2 Mechanical Keyboard ["Electronics", "Accessories"] Electronics 2 -Shirt ["Clothing", "Office"] Clothing 2 -Shirt ["Clothing", "Office"] Office 2 +Shirt ["Clothing", "Office", "Shirt"] Clothing 3 +Shirt ["Clothing", "Office", "Shirt"] Office 3 +Shirt ["Clothing", "Office", "Shirt"] Shirt 3 -- !sql_sum_window -- -Badminton Racket [3] 3 6 -Basketball [3] 3 6 -Laptop [1, 2] 1 3 -Laptop [1, 2] 2 3 -Mechanical Keyboard [1] 1 1 +Badminton Racket [3] 3 3 +Basketball [1, 3] 1 4 +Basketball [1, 3] 3 4 +Laptop [1, 2, 3] 1 6 +Laptop [1, 2, 3] 2 6 +Laptop [1, 2, 3] 3 6 +Mechanical Keyboard [1, 2] 1 3 +Mechanical Keyboard [1, 2] 2 3 Shirt [4] 4 4 --- !sql_join_complex -- +-- !sql_inner_join0 -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop +5 Shirt ["Clothing", "Office", "Shirt"] Shirt + +-- !sql_inner_join1 -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Electronics +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] High-End +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Office +3 Basketball ["Sports", "Outdoor"] Outdoor +3 Basketball ["Sports", "Outdoor"] Sports + +-- !sql_inner_join2 -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Electronics +2 Mechanical Keyboard ["Electronics", "Accessories"] Electronics +3 Basketball ["Sports", "Outdoor"] Sports +4 Badminton Racket ["Sports", "Equipment"] Sports + +-- !sql_left_join0 -- 1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop 2 Mechanical Keyboard ["Electronics", "Accessories"] \N 3 Basketball ["Sports", "Outdoor"] \N 4 Badminton Racket ["Sports", "Equipment"] \N -5 Shirt ["Clothing", "Office"] \N +5 Shirt ["Clothing", "Office", "Shirt"] Shirt + +-- !sql_left_join1 -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Electronics +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] High-End +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Office +2 Mechanical Keyboard ["Electronics", "Accessories"] \N +3 Basketball ["Sports", "Outdoor"] Outdoor +3 Basketball ["Sports", "Outdoor"] Sports +4 Badminton Racket ["Sports", "Equipment"] \N +5 Shirt ["Clothing", "Office", "Shirt"] \N + +-- !sql_left_join2 -- +1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Electronics +2 Mechanical Keyboard ["Electronics", "Accessories"] Electronics +3 Basketball ["Sports", "Outdoor"] Sports +4 Badminton Racket ["Sports", "Equipment"] Sports +5 Shirt ["Clothing", "Office", "Shirt"] \N + +-- !big_array_unnest_all -- +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] + +-- !big_array_unnest0 -- +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 1 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 2 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 3 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 4 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 5 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 6 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 7 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 8 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 9 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 10 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 11 +1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 12 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 3 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 4 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 5 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 6 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 7 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 8 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 9 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 10 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 11 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 12 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 13 +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 14 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 4 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 5 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 6 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 7 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 8 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 9 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 10 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 11 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 12 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 13 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 14 +3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 15 + +-- !big_array_unnest_inner0 -- +1 1 +3 3 + +-- !big_array_unnest_outer0 -- +1 1 +2 \N +3 3 diff --git a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy index 8efc120ab9a612..55bad54f3e7015 100644 --- a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy +++ b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy @@ -117,11 +117,11 @@ suite("nereids_unnest_fn") { ); INSERT INTO items_dict_unnest_t (id, name, tags, price, category_ids) VALUES - (1, 'Laptop', ['Electronics', 'Office', 'High-End', 'Laptop'], 5999.99, [1, 2]), - (2, 'Mechanical Keyboard', ['Electronics', 'Accessories'], 399.99, [1]), - (3, 'Basketball', ['Sports', 'Outdoor'], 199.99, [3]), + (1, 'Laptop', ['Electronics', 'Office', 'High-End', 'Laptop'], 5999.99, [1, 2, 3]), + (2, 'Mechanical Keyboard', ['Electronics', 'Accessories'], 399.99, [1, 2]), + (3, 'Basketball', ['Sports', 'Outdoor'], 199.99, [1,3]), (4, 'Badminton Racket', ['Sports', 'Equipment'], 299.99, [3]), - (5, 'Shirt', ['Clothing', 'Office'], 259.00, [4]); + (5, 'Shirt', ['Clothing', 'Office', 'Shirt'], 259.00, [4]); INSERT INTO categories_dict_unnest_t (id, name) VALUES (1, 'Digital Products'), @@ -354,7 +354,38 @@ suite("nereids_unnest_fn") { name, category_ids, unnest(category_ids); ''' - order_qt_sql_join_complex ''' + order_qt_sql_inner_join0 ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join1 ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON name in ('Laptop', 'Basketball'); + ''' + order_qt_sql_inner_join2 ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON tag in ('Electronics', 'Sports'); + ''' + + order_qt_sql_left_join0 ''' SELECT id, name, @@ -364,4 +395,95 @@ suite("nereids_unnest_fn") { items_dict_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; ''' + order_qt_sql_left_join1 ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON name in ('Laptop', 'Basketball'); + ''' + order_qt_sql_left_join2 ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON tag in ('Electronics', 'Sports'); + ''' + + test { + sql ''' + SELECT + id, + name, + tags, + t.tag + FROM + items_dict_unnest_t + right JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + exception "must be INNER, LEFT or CROSS" + } + + // test big array + sql 'use regression_test_nereids_function_p0_gen_function' + sql """ + set batch_size = 10; + """ + multi_sql ''' + DROP TABLE if exists big_array_unnest_t; + CREATE TABLE big_array_unnest_t ( + id INT, + tags ARRAY + ) properties ( + "replication_num" = "1" + ); + ''' + sql ''' + insert into big_array_unnest_t values + (1, [1,2,3,4,5,6,7,8,9,10,11,12]), + (2, [3,4,5,6,7,8,9,10,11,12,13,14]), + (3, [3,4,5,6,7,8,9,10,11,12,13,14,15]); + ''' + /* + streamLoad { + table "big_array_unnest_t" + file """big_array.csv""" + set 'column_separator', '|' + set 'strict_mode', 'true' + set 'max_filter_ratio', '0' + + check { result, exception, startTime, endTime -> + if (exception != null) { + throw exception + } + def json = parseJson(result) + assertEquals("success", json.Status.toLowerCase()) + assertEquals(4, json.NumberTotalRows) + assertEquals(4, json.NumberLoadedRows) + assertEquals(0, json.NumberFilteredRows) + } + } + */ + qt_big_array_unnest_all ''' + select id, array_size(tags), array_sort(tags) from big_array_unnest_t order by id; + ''' + + qt_big_array_unnest0 ''' + select id, array_size(tags), array_sort(tags), unnest(tags) as tag from big_array_unnest_t order by id, tag; + ''' + + qt_big_array_unnest_inner0 ''' + select id, t.tag from big_array_unnest_t INNER JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + qt_big_array_unnest_outer0 ''' + select id, t.tag from big_array_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + } From 938ecd404e6ad650865eb7911d6791d88c0682f9 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Wed, 17 Dec 2025 12:34:08 +0800 Subject: [PATCH 07/14] improve --- .../pipeline/exec/table_function_operator.cpp | 207 +++++++++----- .../pipeline/exec/table_function_operator.h | 9 +- .../gen_function/unnest.out | 252 +++++++++++++++--- .../gen_function/unnest.groovy | 114 ++++++-- 4 files changed, 451 insertions(+), 131 deletions(-) diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 0a53d858d5c93d..615de7c9f7dc4d 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -78,10 +78,8 @@ Status TableFunctionLocalState::open(RuntimeState* state) { return Status::OK(); } -void TableFunctionLocalState::_copy_output_slots( - std::vector& columns, const TableFunctionOperatorX& p, - size_t output_row_count, std::vector& child_row_to_output_rows_indices, - std::vector& handled_row_indices) { +void TableFunctionLocalState::_copy_output_slots(std::vector& columns, + const TableFunctionOperatorX& p) { if (!_current_row_insert_times) { return; } @@ -90,11 +88,6 @@ void TableFunctionLocalState::_copy_output_slots( columns[index]->insert_many_from(*src_column, _cur_child_offset, _current_row_insert_times); } _current_row_insert_times = 0; - - if (_need_to_handle_outer_conjuncts) { - handled_row_indices.push_back(_cur_child_offset); - child_row_to_output_rows_indices.push_back(output_row_count); - } } // Returns the index of fn of the last eos counted from back to front @@ -168,40 +161,33 @@ bool TableFunctionLocalState::_is_inner_and_empty() { Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, vectorized::Block* output_block, bool* eos) { + SCOPED_TIMER(_process_rows_timer); + if (_need_to_handle_outer_conjuncts) { + return _get_expanded_block_for_outer_conjuncts(state, output_block, eos); + } + auto& p = _parent->cast(); vectorized::MutableBlock m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block( output_block, p._output_slots); vectorized::MutableColumns& columns = m_block.mutable_columns(); - auto child_slot_count = p._child_slots.size(); + for (int i = 0; i < p._fn_num; i++) { - if (columns[i + child_slot_count]->is_nullable()) { + if (columns[i + p._child_slots.size()]->is_nullable()) { _fns[i]->set_nullable(); } } - std::vector child_row_to_output_rows_indices; - std::vector handled_row_indices; - bool child_block_empty = _child_block->empty(); - if (_need_to_handle_outer_conjuncts && !child_block_empty) { - child_row_to_output_rows_indices.push_back(0); - } - SCOPED_TIMER(_process_rows_timer); - auto batch_size = state->batch_size(); - int32_t max_batch_size = std::numeric_limits::max(); - while (columns[child_slot_count]->size() < batch_size) { + while (columns[p._child_slots.size()]->size() < state->batch_size()) { RETURN_IF_CANCELLED(state); - // finished handling current child block - if (_child_block->rows() == 0 || _cur_child_offset == -1) { + if (_child_block->rows() == 0) { break; } bool skip_child_row = false; - size_t cur_row_count = 0; - while (cur_row_count = columns[child_slot_count]->size(), cur_row_count < batch_size) { + while (columns[p._child_slots.size()]->size() < state->batch_size()) { int idx = _find_last_fn_eos_idx(); if (idx == 0 || skip_child_row) { - _copy_output_slots(columns, p, cur_row_count, child_row_to_output_rows_indices, - handled_row_indices); + _copy_output_slots(columns, p); // all table functions' results are exhausted, process next child row. process_next_child_row(); if (_cur_child_offset == -1) { @@ -223,35 +209,109 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, DCHECK_LE(1, p._fn_num); // It may take multiple iterations of this while loop to process a child row if // any table function produces a large number of rows. - // But for table function with outer conjuncts, we output all rows in one iteration - // even if it may exceed batch_size, because we need to make sure at least one row - // is output for the child row after evaluating the outer conjuncts. auto repeat_times = _fns[p._fn_num - 1]->get_value( - columns[child_slot_count + p._fn_num - 1], + columns[p._child_slots.size() + p._fn_num - 1], //// It has already been checked that // columns[p._child_slots.size()]->size() < state->batch_size(), // so columns[p._child_slots.size()]->size() will not exceed the range of int. - _need_to_handle_outer_conjuncts - ? max_batch_size - : batch_size - (int)columns[child_slot_count]->size()); + state->batch_size() - (int)columns[p._child_slots.size()]->size()); _current_row_insert_times += repeat_times; for (int i = 0; i < p._fn_num - 1; i++) { - LOG(INFO) << "xxxx Query: " << print_id(state->query_id()) - << " get_same_many_values for fn index: " << i; - _fns[i]->get_same_many_values(columns[i + child_slot_count], repeat_times); + _fns[i]->get_same_many_values(columns[i + p._child_slots.size()], repeat_times); } } } - _copy_output_slots(columns, p, columns[child_slot_count]->size(), - child_row_to_output_rows_indices, handled_row_indices); + _copy_output_slots(columns, p); - size_t row_size = columns[child_slot_count]->size(); + size_t row_size = columns[p._child_slots.size()]->size(); for (auto index : p._useless_slot_indexs) { columns[index]->insert_many_defaults(row_size - columns[index]->size()); } - // LOG(INFO) << "xxxx before filter output_block data: " - // << output_block->dump_data(0, output_block->rows()); + + { + SCOPED_TIMER(_filter_timer); // 3. eval conjuncts + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_expand_conjuncts_ctxs, output_block, + output_block->columns())); + RETURN_IF_ERROR(vectorized::VExprContext::filter_block(_conjuncts, output_block, + output_block->columns())); + } + + *eos = _child_eos && _cur_child_offset == -1; + return Status::OK(); +} + +Status TableFunctionLocalState::_get_expanded_block_for_outer_conjuncts( + RuntimeState* state, vectorized::Block* output_block, bool* eos) { + auto& p = _parent->cast(); + vectorized::MutableBlock m_block = vectorized::VectorizedUtils::build_mutable_mem_reuse_block( + output_block, p._output_slots); + vectorized::MutableColumns& columns = m_block.mutable_columns(); + auto child_slot_count = p._child_slots.size(); + for (int i = 0; i < p._fn_num; i++) { + if (columns[i + child_slot_count]->is_nullable()) { + _fns[i]->set_nullable(); + } + } + + std::vector child_row_to_output_rows_indices; + std::vector handled_row_indices; + bool child_block_empty = _child_block->empty(); + if (!child_block_empty) { + child_row_to_output_rows_indices.push_back(0); + } + + auto batch_size = state->batch_size(); + auto output_row_count = columns[child_slot_count]->size(); + while (output_row_count < batch_size) { + RETURN_IF_CANCELLED(state); + + // finished handling current child block + if (_cur_child_offset == -1) { + break; + } + + bool skip_child_row = false; + while (output_row_count < batch_size) { + // if table function is not outer and has empty result, go to next child row + if (_fns[0]->eos() || skip_child_row) { + _copy_output_slots(columns, p); + if (!skip_child_row) { + handled_row_indices.push_back(_cur_child_offset); + child_row_to_output_rows_indices.push_back(output_row_count); + } + process_next_child_row(); + if (_cur_child_offset == -1) { + break; + } + } + if (skip_child_row = _is_inner_and_empty(); skip_child_row) { + _child_rows_has_output[_cur_child_offset] = true; + continue; + } + + // It may take multiple iterations of this while loop to process a child row if + // the table function produces a large number of rows. + auto repeat_times = _fns[0]->get_value(columns[child_slot_count], + batch_size - (int)output_row_count); + _current_row_insert_times += repeat_times; + output_row_count = columns[child_slot_count]->size(); + } + } + // Two scenarios the loop above will exit: + // 1. current child block is finished processing + // _cur_child_offset == -1 + // 2. output_block reaches batch size + // fn maybe or maybe not eos + if (output_row_count >= batch_size) { + _copy_output_slots(columns, p); + handled_row_indices.push_back(_cur_child_offset); + child_row_to_output_rows_indices.push_back(output_row_count); + } + for (auto index : p._useless_slot_indexs) { + columns[index]->insert_many_defaults(output_row_count - columns[index]->size()); + } + output_block->set_columns(std::move(columns)); /** Handle the outer conjuncts after unnest. Currently, only left outer is supported. @@ -314,8 +374,7 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, ] child rows 1, 2, 3 are all filtered out, so we need to insert one row with NULL tag value for each of them. */ - if (!_expand_conjuncts_ctxs.empty() && !child_block_empty) { - auto output_row_count = output_block->rows(); + if (!child_block_empty) { vectorized::IColumn::Filter filter; auto column_count = output_block->columns(); vectorized::ColumnNumbers columns_to_filter(column_count); @@ -323,44 +382,45 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, RETURN_IF_ERROR(vectorized::VExprContext::execute_conjuncts_and_filter_block( _expand_conjuncts_ctxs, output_block, columns_to_filter, column_count, filter)); size_t remain_row_count = output_block->rows(); - /* - LOG(INFO) << "xxxx after filter output_block row count: " << remain_row_count; - std::string debug_string("xxxx handled child rows: "); - for (auto v : handled_row_indices) { - debug_string += std::to_string(v) + ","; - } - debug_string += ", filter: "; - for (auto v : filter) { - debug_string += std::to_string(v) + ","; - } - debug_string += ", child_row_to_output_rows_indices: "; - for (auto v : child_row_to_output_rows_indices) { - debug_string += ", " + std::to_string(v); - } - LOG(INFO) << debug_string; - */ // for outer table function, need to handle those child rows which all expanded rows are filtered out - if (_need_to_handle_outer_conjuncts && remain_row_count < output_row_count) { - auto handled_child_row_count = handled_row_indices.size(); + auto handled_child_row_count = handled_row_indices.size(); + if (remain_row_count < output_row_count) { for (size_t i = 0; i < handled_child_row_count; ++i) { auto start_row_idx = child_row_to_output_rows_indices[i]; auto end_row_idx = child_row_to_output_rows_indices[i + 1]; - if (!simd::contain_byte((uint8_t*)filter.data() + start_row_idx, - end_row_idx - start_row_idx, 1)) { + if (simd::contain_byte((uint8_t*)filter.data() + start_row_idx, + end_row_idx - start_row_idx, 1)) { + _child_rows_has_output[handled_row_indices[i]] = true; + } + } + } else { + for (auto row_idx : handled_row_indices) { + _child_rows_has_output[row_idx] = true; + } + } + + if (-1 == _cur_child_offset) { + // Finished handling current child block, + vectorized::MutableBlock m_block2 = + vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, + p._output_slots); + vectorized::MutableColumns& columns2 = m_block2.mutable_columns(); + auto child_block_row_count = _child_block->rows(); + for (size_t i = 0; i != child_block_row_count; i++) { + if (!_child_rows_has_output[i]) { + // insert one row with NULL for this child row for (auto index : p._output_slot_indexs) { auto src_column = _child_block->get_by_position(index).column; - columns[index]->insert_from(*src_column, handled_row_indices[i]); + columns2[index]->insert_from(*src_column, i); } for (auto index : p._useless_slot_indexs) { - columns[index]->insert_default(); - } - for (int j = 0; j != p._fn_num; j++) { - columns[j + child_slot_count]->insert_default(); + columns2[index]->insert_default(); } + columns2[child_slot_count]->insert_default(); } } - } - if (_cur_child_offset == -1) { + output_block->set_columns(std::move(columns2)); + _child_rows_has_output.clear(); _child_block->clear_column_data(_parent->cast() ._child->row_desc() .num_materialized_slots()); @@ -374,7 +434,6 @@ Status TableFunctionLocalState::get_expanded_block(RuntimeState* state, } *eos = _child_eos && _cur_child_offset == -1; - // LOG(INFO) << "xxxx output_block rows: " << output_block->rows(); return Status::OK(); } @@ -387,9 +446,9 @@ void TableFunctionLocalState::process_next_child_row() { fn->process_close(); } - // if there are any _expand_conjuncts_ctxs, don't clear child block here, + // if there are any _expand_conjuncts_ctxs and it's outer, don't clear child block here, // because we still need _child_block to output NULL rows for outer table function - if (_expand_conjuncts_ctxs.empty()) { + if (!_need_to_handle_outer_conjuncts) { _child_block->clear_column_data(_parent->cast() ._child->row_desc() .num_materialized_slots()); diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index 61f48469a68f97..fdbde3b46fd9c9 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -57,9 +57,7 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> MOCK_FUNCTION Status _clone_table_function(RuntimeState* state); void _copy_output_slots(std::vector& columns, - const TableFunctionOperatorX& p, size_t output_row_count, - std::vector& child_row_to_output_rows_indices, - std::vector& handled_row_indices); + const TableFunctionOperatorX& p); bool _roll_table_functions(int last_eos_idx); // return: // 0: all fns are eos @@ -68,6 +66,9 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> int _find_last_fn_eos_idx() const; bool _is_inner_and_empty(); + Status _get_expanded_block_for_outer_conjuncts(RuntimeState* state, + vectorized::Block* output_block, bool* eos); + std::vector _fns; vectorized::VExprContextSPtrs _vfn_ctxs; vectorized::VExprContextSPtrs _expand_conjuncts_ctxs; @@ -77,6 +78,7 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> std::unique_ptr _child_block; int _current_row_insert_times = 0; bool _child_eos = false; + std::vector _child_rows_has_output; RuntimeProfile::Counter* _init_function_timer = nullptr; RuntimeProfile::Counter* _process_rows_timer = nullptr; @@ -116,6 +118,7 @@ class TableFunctionOperatorX MOCK_REMOVE(final) SCOPED_TIMER(local_state._init_function_timer); RETURN_IF_ERROR(fn->process_init(input_block, state)); } + local_state._child_rows_has_output.resize(input_block->rows(), false); local_state.process_next_child_row(); return Status::OK(); } diff --git a/regression-test/data/nereids_function_p0/gen_function/unnest.out b/regression-test/data/nereids_function_p0/gen_function/unnest.out index 268d7984e6d0f6..4dfe4464a93482 100644 --- a/regression-test/data/nereids_function_p0/gen_function/unnest.out +++ b/regression-test/data/nereids_function_p0/gen_function/unnest.out @@ -614,23 +614,118 @@ Shirt [4] 4 4 5 Shirt ["Clothing", "Office", "Shirt"] \N -- !big_array_unnest_all -- -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] -2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] +3 2 [1, 2] +4 2 [4, 4] +5 \N \N +6 5 [null, null, 1, 2, 5] +7 2 [1, 7] +8 2 [1, 9] +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] -- !big_array_unnest0 -- -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 1 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 2 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 3 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 4 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 5 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 6 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 7 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 8 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 9 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 10 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 11 -1 12 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12] 12 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 1 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 1 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 2 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 4 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 5 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 6 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 7 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 8 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 9 +1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 10 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 1 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 3 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 3 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 4 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 5 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 6 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 7 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 8 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 9 +2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] 10 +3 2 [1, 2] 1 +3 2 [1, 2] 2 +4 2 [4, 4] 4 +4 2 [4, 4] 4 +6 5 [null, null, 1, 2, 5] \N +6 5 [null, null, 1, 2, 5] \N +6 5 [null, null, 1, 2, 5] 1 +6 5 [null, null, 1, 2, 5] 2 +6 5 [null, null, 1, 2, 5] 5 +7 2 [1, 7] 1 +7 2 [1, 7] 7 +8 2 [1, 9] 1 +8 2 [1, 9] 9 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 1 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 1 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 2 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 2 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 3 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 3 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 4 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 4 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 5 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 5 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 6 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 6 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 7 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 8 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 9 +9 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 9] 9 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 1 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 1 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 2 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 2 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 3 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 3 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 4 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 4 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 5 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 5 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 6 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 6 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 7 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 8 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 9 +10 16 [1, 1, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 8, 9, 10] 10 + +-- !big_array_unnest_inner0 -- +1 1 +1 1 +4 4 +4 4 +7 7 +9 9 +9 9 +10 10 + +-- !big_array_unnest_outer0 -- +1 1 +1 1 +2 \N +3 \N +4 4 +4 4 +5 \N +6 \N +7 7 +8 \N +9 9 +9 9 +10 10 + +-- !big_array_unnest_all_1 -- +1 3 [1, 2, 3] +2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] + +-- !big_array_unnest1 -- +1 3 [1, 2, 3] 1 +1 3 [1, 2, 3] 2 +1 3 [1, 2, 3] 3 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 3 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 4 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 5 @@ -643,26 +738,123 @@ Shirt [4] 4 4 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 12 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 13 2 12 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] 14 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 4 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 5 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 6 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 7 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 8 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 9 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 10 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 11 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 12 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 13 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 14 -3 13 [3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 15 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 4 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 5 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 6 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 7 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 8 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 9 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 10 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 11 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 12 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 13 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 14 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 15 + +-- !big_array_unnest_inner1 -- +1 1 +3 3 +3 3 --- !big_array_unnest_inner0 -- +-- !big_array_unnest_outer1 -- 1 1 +2 \N +3 3 3 3 --- !big_array_unnest_outer0 -- +-- !big_array_unnest_all_2 -- +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] +2 3 [1, 3, 4] +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] + +-- !big_array_unnest2 -- +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 1 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 2 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 3 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 4 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 5 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 6 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 7 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 8 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 9 +1 10 [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] 10 +2 3 [1, 3, 4] 1 +2 3 [1, 3, 4] 3 +2 3 [1, 3, 4] 4 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 4 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 5 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 6 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 7 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 8 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 9 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 10 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 11 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 12 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 13 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 14 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 15 + +-- !big_array_unnest_inner2 -- +1 1 +3 3 +3 3 + +-- !big_array_unnest_outer2 -- 1 1 2 \N 3 3 +3 3 + +-- !big_array_unnest_all_3 -- +1 3 [1, 2, 3] +2 3 [1, 3, 4] +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] +4 0 [] +5 \N \N +6 4 [null, 1, 2, 6] + +-- !big_array_unnest3 -- +1 3 [1, 2, 3] 1 +1 3 [1, 2, 3] 2 +1 3 [1, 2, 3] 3 +2 3 [1, 3, 4] 1 +2 3 [1, 3, 4] 3 +2 3 [1, 3, 4] 4 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 3 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 4 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 5 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 6 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 7 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 8 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 9 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 10 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 11 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 12 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 13 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 14 +3 14 [3, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15] 15 +6 4 [null, 1, 2, 6] \N +6 4 [null, 1, 2, 6] 1 +6 4 [null, 1, 2, 6] 2 +6 4 [null, 1, 2, 6] 6 + +-- !big_array_unnest_inner3 -- +1 1 +3 3 +3 3 +6 6 + +-- !big_array_unnest_outer3 -- +1 1 +2 \N +3 3 +3 3 +4 \N +5 \N +6 6 diff --git a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy index 55bad54f3e7015..7632c066fb1f38 100644 --- a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy +++ b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy @@ -444,32 +444,20 @@ suite("nereids_unnest_fn") { "replication_num" = "1" ); ''' + // one child row unnest to batch_size or more output rows sql ''' insert into big_array_unnest_t values - (1, [1,2,3,4,5,6,7,8,9,10,11,12]), - (2, [3,4,5,6,7,8,9,10,11,12,13,14]), - (3, [3,4,5,6,7,8,9,10,11,12,13,14,15]); - ''' - /* - streamLoad { - table "big_array_unnest_t" - file """big_array.csv""" - set 'column_separator', '|' - set 'strict_mode', 'true' - set 'max_filter_ratio', '0' - - check { result, exception, startTime, endTime -> - if (exception != null) { - throw exception - } - def json = parseJson(result) - assertEquals("success", json.Status.toLowerCase()) - assertEquals(4, json.NumberTotalRows) - assertEquals(4, json.NumberLoadedRows) - assertEquals(0, json.NumberFilteredRows) - } - } - */ + (1, [1,2,1,4,5,6,7,8,9,10]), + (2, [1,3,3,4,5,6,7,8,9,10]), + (3, [1,2]), + (4, [4,4]), + (5, null), + (6, [1,null,2,null,5]), + (7, [1, 7]), + (8, [1, 9]), + (9, [1,2,3,4,5,6, 1,2,3,4,5,6,7,8,9,9]), + (10, [1,2,3,4,5,6, 1,2,3,4,5,6,7,8,9,10]); + ''' qt_big_array_unnest_all ''' select id, array_size(tags), array_sort(tags) from big_array_unnest_t order by id; ''' @@ -486,4 +474,82 @@ suite("nereids_unnest_fn") { select id, t.tag from big_array_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; ''' + sql """ + truncate table big_array_unnest_t; + """ + sql ''' + insert into big_array_unnest_t values + (1, [1,2,3]), + (2, [3,4,5,6,7,8,9,10,11,12,13,14]), + (3, [3,3,4,5,6,7,8,9,10,11,12,13,14,15]); + ''' + qt_big_array_unnest_all_1 ''' + select id, array_size(tags), array_sort(tags) from big_array_unnest_t order by id; + ''' + + qt_big_array_unnest1 ''' + select id, array_size(tags), array_sort(tags), unnest(tags) as tag from big_array_unnest_t order by id, tag; + ''' + + qt_big_array_unnest_inner1 ''' + select id, t.tag from big_array_unnest_t INNER JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + qt_big_array_unnest_outer1 ''' + select id, t.tag from big_array_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + sql """ + truncate table big_array_unnest_t; + """ + sql ''' + insert into big_array_unnest_t values + (1, [1, 2, 3,4,5,6,7,8,9,10]), + (2, [1,3,4]), + (3, [3,3,4,5,6,7,8,9,10,11,12,13,14,15]); + ''' + qt_big_array_unnest_all_2 ''' + select id, array_size(tags), array_sort(tags) from big_array_unnest_t order by id; + ''' + + qt_big_array_unnest2 ''' + select id, array_size(tags), array_sort(tags), unnest(tags) as tag from big_array_unnest_t order by id, tag; + ''' + + qt_big_array_unnest_inner2 ''' + select id, t.tag from big_array_unnest_t INNER JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + qt_big_array_unnest_outer2 ''' + select id, t.tag from big_array_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + sql """ + truncate table big_array_unnest_t; + """ + sql ''' + insert into big_array_unnest_t values + (1, [1, 2, 3]), + (2, [1,3,4]), + (3, [3,3,4,5,6,7,8,9,10,11,12,13,14,15]), + (4, []), + (5, null), + (6, [1,null,2, 6]); + ''' + qt_big_array_unnest_all_3 ''' + select id, array_size(tags), array_sort(tags) from big_array_unnest_t order by id; + ''' + + qt_big_array_unnest3 ''' + select id, array_size(tags), array_sort(tags), unnest(tags) as tag from big_array_unnest_t order by id, tag; + ''' + + qt_big_array_unnest_inner3 ''' + select id, t.tag from big_array_unnest_t INNER JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + + qt_big_array_unnest_outer3 ''' + select id, t.tag from big_array_unnest_t LEFT JOIN lateral unnest(tags) AS t(tag) ON tag = id order by id, tag; + ''' + } From c8d042d621c2f42f5aa1e6b86c8c3f04b24ea7e0 Mon Sep 17 00:00:00 2001 From: lichi Date: Thu, 18 Dec 2025 15:39:04 +0800 Subject: [PATCH 08/14] fix bug --- .../nereids/glue/translator/PhysicalPlanTranslator.java | 5 +++++ .../java/org/apache/doris/planner/TableFunctionNode.java | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 38e5976dd7349b..9024639b139d27 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -2153,6 +2153,11 @@ public PlanFragment visitPhysicalProject(PhysicalProject project if (inputPlanNode instanceof TableFunctionNode) { TableFunctionNode tableFunctionNode = (TableFunctionNode) inputPlanNode; + // slots used by expandConjuncts must be added to TableFunctionNode's output slot ids + List expandConjuncts = tableFunctionNode.getExpandConjuncts(); + for (Expr expr : expandConjuncts) { + Expr.extractSlots(expr, requiredSlotIdSet); + } tableFunctionNode.setOutputSlotIds(Lists.newArrayList(requiredSlotIdSet)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java index a5099999366238..5195c1f3a06db7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java @@ -64,6 +64,10 @@ public void setOutputSlotIds(List outputSlotIds) { this.outputSlotIds = outputSlotIds; } + public List getExpandConjuncts() { + return expandConjuncts; + } + @Override public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { StringBuilder output = new StringBuilder(); From 9bc84c820f0e3729335b4f30e97146155540fdc7 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Thu, 18 Dec 2025 15:41:04 +0800 Subject: [PATCH 09/14] add be ut and p0 --- .../pipeline/exec/table_function_operator.cpp | 3 + .../operator/table_function_operator_test.cpp | 576 ++++++++++++++++++ .../gen_function/unnest.out | 66 ++ .../gen_function/unnest.groovy | 87 +++ 4 files changed, 732 insertions(+) diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index 615de7c9f7dc4d..ea400a9ee45ea2 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -307,6 +307,9 @@ Status TableFunctionLocalState::_get_expanded_block_for_outer_conjuncts( _copy_output_slots(columns, p); handled_row_indices.push_back(_cur_child_offset); child_row_to_output_rows_indices.push_back(output_row_count); + if (_fns[0]->eos()) { + process_next_child_row(); + } } for (auto index : p._useless_slot_indexs) { columns[index]->insert_many_defaults(output_row_count - columns[index]->size()); diff --git a/be/test/pipeline/operator/table_function_operator_test.cpp b/be/test/pipeline/operator/table_function_operator_test.cpp index ee555fa4efe6f4..bd4e4fd67ed61f 100644 --- a/be/test/pipeline/operator/table_function_operator_test.cpp +++ b/be/test/pipeline/operator/table_function_operator_test.cpp @@ -17,6 +17,9 @@ #include "pipeline/exec/table_function_operator.h" +#include +#include +#include #include #include @@ -24,8 +27,10 @@ #include "operator_helper.h" #include "testutil/column_helper.h" +#include "testutil/creators.h" #include "testutil/mock/mock_descriptors.h" #include "testutil/mock/mock_literal_expr.h" +#include "testutil/mock/mock_operators.h" #include "testutil/mock/mock_slot_ref.h" #include "vec/core/block.h" namespace doris::pipeline { @@ -355,4 +360,575 @@ TEST_F(TableFunctionOperatorTest, single_two_eos_test) { } } +struct UnnestTest : public ::testing::Test { + void SetUp() override { + type_node_bool.type = TTypeNodeType::SCALAR; + type_node_bool.scalar_type.type = TPrimitiveType::BOOLEAN; + type_node_bool.__isset.scalar_type = true; + + type_node_int.type = TTypeNodeType::SCALAR; + type_node_int.scalar_type.type = TPrimitiveType::INT; + type_node_int.__isset.scalar_type = true; + + type_node_arr.type = TTypeNodeType::ARRAY; + type_node_arr.contains_null = true; + type_node_arr.__isset.contains_nulls = true; + type_node_arr.contains_nulls.emplace_back(true); + + type_desc_int.types.emplace_back(type_node_int); + type_desc_int.byte_size = -1; + + type_desc_array_int.types.emplace_back(type_node_arr); + type_desc_array_int.types.emplace_back(type_node_int); + type_desc_array_int.byte_size = -1; + + type_desc_boolean.types.emplace_back(type_node_bool); + type_desc_boolean.byte_size = -1; + + // input block tuple + tuple0.id = tuple_id++; + tuple0.byteSize = 0; + tuple0.numNullBytes = 0; + tuple0.tableId = 0; + + // unnested output tuple + tuple1.id = tuple_id++; + tuple1.byteSize = 0; + tuple1.numNullBytes = 0; + + // projection tuple for unnest + tuple2.id = tuple_id++; + tuple2.byteSize = 0; + tuple2.numNullBytes = 0; + + // SlotRefs for input slots + // `id` column, type int + slot_ref_id.slot_id = slot_id++; + slot_ref_id.tuple_id = tuple0.id; + slot_ref_id.col_unique_id = col_unique_id++; + slot_ref_id.is_virtual_slot = false; + + // `tags` column, type array + slot_ref_tags.slot_id = slot_id++; + slot_ref_tags.tuple_id = tuple0.id; + slot_ref_tags.col_unique_id = col_unique_id++; + slot_ref_tags.is_virtual_slot = false; + + // unnested output slot, tag + slot_ref_unnest_tag.slot_id = slot_id++; + slot_ref_unnest_tag.tuple_id = tuple1.id; + slot_ref_unnest_tag.col_unique_id = -1; + slot_ref_unnest_tag.is_virtual_slot = false; + + slot_desc_id.id = slot_ref_id.slot_id; + slot_desc_id.parent = tuple0.id; + slot_desc_id.slotType = type_desc_int; + slot_desc_id.columnPos = -1; + slot_desc_id.byteOffset = 0; + slot_desc_id.nullIndicatorByte = 0; + slot_desc_id.nullIndicatorBit = 0; + slot_desc_id.colName = "id"; + slot_desc_id.slotIdx = -1; + slot_desc_id.isMaterialized = true; + slot_desc_id.col_unique_id = slot_ref_id.col_unique_id; + slot_desc_id.is_key = true; + slot_desc_id.need_materialize = true; + slot_desc_id.is_auto_increment = false; + slot_desc_id.primitive_type = TPrimitiveType::INT; + + slot_desc_tags.id = slot_ref_tags.slot_id; + slot_desc_tags.parent = tuple0.id; + slot_desc_tags.slotType = type_desc_array_int; + slot_desc_tags.columnPos = -1; + slot_desc_tags.byteOffset = 0; + slot_desc_tags.nullIndicatorByte = 0; + slot_desc_tags.nullIndicatorBit = 0; + slot_desc_tags.colName = "tags"; + slot_desc_tags.slotIdx = -1; + slot_desc_tags.isMaterialized = true; + slot_desc_tags.col_unique_id = slot_ref_tags.col_unique_id; + slot_desc_tags.is_key = false; + slot_desc_tags.need_materialize = true; + slot_desc_tags.is_auto_increment = false; + slot_desc_tags.primitive_type = TPrimitiveType::ARRAY; + + slot_desc_unnest_tag.id = slot_ref_unnest_tag.slot_id; + slot_desc_unnest_tag.parent = tuple1.id; + slot_desc_unnest_tag.slotType = type_desc_int; + slot_desc_unnest_tag.columnPos = -1; + slot_desc_unnest_tag.byteOffset = 0; + slot_desc_unnest_tag.nullIndicatorByte = 0; + slot_desc_unnest_tag.nullIndicatorBit = 0; + slot_desc_unnest_tag.colName = "tag#3"; + slot_desc_unnest_tag.slotIdx = -1; + slot_desc_unnest_tag.isMaterialized = true; + slot_desc_unnest_tag.col_unique_id = -1; + slot_desc_unnest_tag.is_key = false; + slot_desc_unnest_tag.need_materialize = true; + slot_desc_unnest_tag.is_auto_increment = false; + slot_desc_unnest_tag.primitive_type = TPrimitiveType::INT; + + slot_desc_id_2 = slot_desc_id; + slot_desc_id_2.id = slot_id++; + slot_desc_id_2.parent = tuple2.id; + + slot_desc_unnest_tag2 = slot_desc_unnest_tag; + slot_desc_unnest_tag2.id = slot_id++; + slot_desc_unnest_tag2.parent = tuple2.id; + } + void setup_exec_env() { + runtime_state = std::make_unique(); + obj_pool = std::make_unique(); + query_ctx = generate_one_query(); + runtime_profile = std::make_shared("test"); + runtime_state->batsh_size = 5; + runtime_state->_query_ctx = query_ctx.get(); + runtime_state->_query_id = query_ctx->query_id(); + runtime_state->resize_op_id_to_local_state(-100); + runtime_state->set_max_operator_id(-100); + } + + void setup_plan_node(TPlanNode& tplan_node_table_function_node, bool outer) { + // Set main TPlanNode properties + tplan_node_table_function_node.node_id = 0; + tplan_node_table_function_node.node_type = TPlanNodeType::TABLE_FUNCTION_NODE; + tplan_node_table_function_node.num_children = 1; + tplan_node_table_function_node.limit = -1; + + tplan_node_table_function_node.row_tuples.push_back(tuple0.id); + tplan_node_table_function_node.row_tuples.push_back(tuple1.id); + tplan_node_table_function_node.nullable_tuples.push_back(false); + tplan_node_table_function_node.nullable_tuples.push_back(false); + tplan_node_table_function_node.compact_data = false; + tplan_node_table_function_node.is_serial_operator = false; + + // setup table function node + tplan_node_table_function_node.__set_table_function_node(TTableFunctionNode()); + + // setup fnCallExprList of table function node + TExpr fn_expr; + + fn_expr.nodes.emplace_back(); + fn_expr.nodes[0].node_type = TExprNodeType::FUNCTION_CALL; + fn_expr.nodes[0].type.types.emplace_back(type_node_int); + fn_expr.nodes[0].num_children = 1; + + // setup TFunction of table function node + fn_expr.nodes[0].__set_fn(TFunction()); + fn_expr.nodes[0].fn.__set_name(TFunctionName()); + fn_expr.nodes[0].fn.name.function_name = outer ? "explode_outer" : "explode"; + + fn_expr.nodes[0].fn.arg_types.emplace_back(type_desc_array_int); + fn_expr.nodes[0].fn.ret_type.types.emplace_back(type_node_int); + + fn_expr.nodes[0].fn.has_var_args = false; + fn_expr.nodes[0].fn.signature = outer ? "explode_outer(array)" : "explode(array)"; + fn_expr.nodes[0].fn.__set_scalar_fn(TScalarFunction()); + fn_expr.nodes[0].fn.scalar_fn.symbol = ""; + fn_expr.nodes[0].fn.id = 0; + fn_expr.nodes[0].fn.vectorized = true; + fn_expr.nodes[0].fn.is_udtf_function = false; + fn_expr.nodes[0].fn.is_static_load = false; + fn_expr.nodes[0].fn.expiration_time = 360; + fn_expr.nodes[0].is_nullable = true; + + // explode input slot ref: array + fn_expr.nodes.emplace_back(); + fn_expr.nodes[1].node_type = TExprNodeType::SLOT_REF; + fn_expr.nodes[1].type = type_desc_array_int; + fn_expr.nodes[1].num_children = 0; + fn_expr.nodes[1].__set_slot_ref(TSlotRef()); + fn_expr.nodes[1].slot_ref = slot_ref_tags; + fn_expr.nodes[1].output_scale = -1; + fn_expr.nodes[1].is_nullable = true; + fn_expr.nodes[1].label = "tags"; + + tplan_node_table_function_node.table_function_node.fnCallExprList.push_back(fn_expr); + + // Set output slot IDs + tplan_node_table_function_node.table_function_node.outputSlotIds.push_back(slot_desc_id.id); + tplan_node_table_function_node.table_function_node.outputSlotIds.push_back( + slot_desc_unnest_tag.id); + + // Set expand conjuncts: tag = id + TExpr expand_expr; + { + TExprNode expr_node; + expr_node.node_type = TExprNodeType::BINARY_PRED; + expr_node.type = type_desc_boolean; + + expr_node.opcode = TExprOpcode::EQ; + expr_node.num_children = 2; + expr_node.output_scale = -1; + + expr_node.__set_fn(TFunction()); + expr_node.fn.__set_name(TFunctionName()); + expr_node.fn.name.function_name = "eq"; + expr_node.fn.binary_type = TFunctionBinaryType::BUILTIN; + expr_node.fn.arg_types.push_back(type_desc_int); + expr_node.fn.arg_types.push_back(type_desc_int); + expr_node.fn.ret_type = type_desc_boolean; + expr_node.fn.has_var_args = false; + expr_node.fn.signature = "eq(int, int)"; + expr_node.fn.id = 0; + expr_node.fn.is_udtf_function = false; + + expr_node.child_type = TPrimitiveType::INT; + expr_node.is_nullable = true; + + expand_expr.nodes.emplace_back(expr_node); + } + { + TExprNode expr_node; + expr_node.node_type = TExprNodeType::SLOT_REF; + expr_node.type = type_desc_int; + expr_node.num_children = 0; + + expr_node.slot_ref = slot_ref_unnest_tag; + expr_node.output_scale = -1; + expr_node.is_nullable = true; + expr_node.label = "tag"; + + expand_expr.nodes.emplace_back(expr_node); + } + { + TExprNode expr_node; + expr_node.node_type = TExprNodeType::SLOT_REF; + expr_node.type = type_desc_int; + expr_node.num_children = 0; + + expr_node.slot_ref = slot_ref_id; + expr_node.output_scale = -1; + expr_node.is_nullable = true; + expr_node.label = "id"; + + expand_expr.nodes.emplace_back(expr_node); + } + tplan_node_table_function_node.table_function_node.expand_conjuncts.push_back(expand_expr); + // end of TTableFunctionNode setup + + // Set projections + TExpr texpr_proj0; + texpr_proj0.nodes.emplace_back(); + texpr_proj0.nodes[0].node_type = TExprNodeType::SLOT_REF; + texpr_proj0.nodes[0].type = type_desc_int; + texpr_proj0.nodes[0].num_children = 0; + texpr_proj0.nodes[0].__set_slot_ref(TSlotRef()); + texpr_proj0.nodes[0].slot_ref = slot_ref_id; + texpr_proj0.nodes[0].output_scale = -1; + texpr_proj0.nodes[0].is_nullable = true; + texpr_proj0.nodes[0].label = "id"; + + TExpr texpr_proj1; + texpr_proj1.nodes.emplace_back(); + texpr_proj1.nodes[0].node_type = TExprNodeType::SLOT_REF; + texpr_proj1.nodes[0].type = type_desc_int; + texpr_proj1.nodes[0].num_children = 0; + texpr_proj1.nodes[0].__set_slot_ref(TSlotRef()); + texpr_proj1.nodes[0].slot_ref = slot_ref_unnest_tag; + texpr_proj1.nodes[0].output_scale = -1; + texpr_proj1.nodes[0].is_nullable = true; + texpr_proj1.nodes[0].label = "tag"; + + tplan_node_table_function_node.projections.push_back(texpr_proj0); + tplan_node_table_function_node.projections.push_back(texpr_proj1); + tplan_node_table_function_node.output_tuple_id = tuple2.id; + tplan_node_table_function_node.nereids_id = 144; + // end of tplan_node_table_function_node + } + + std::shared_ptr create_test_operators( + DescriptorTbl* desc_tbl, const TPlanNode& tplan_node_table_function_node) { + runtime_state->set_desc_tbl(desc_tbl); + RowDescriptor mock_row_desc(runtime_state->desc_tbl(), {tuple0.id}); + LocalStateInfo local_state_info {.parent_profile = runtime_profile.get(), + .scan_ranges = {}, + .shared_state = nullptr, + .shared_state_map = {}, + .task_idx = 0}; + PipelineXLocalStateBase* local_state = nullptr; + std::shared_ptr table_func_op; + std::shared_ptr mock_source_operator; + + table_func_op = std::make_shared( + obj_pool.get(), tplan_node_table_function_node, 0, *desc_tbl); + + mock_source_operator = std::make_shared(); + mock_source_operator->_row_descriptor = mock_row_desc; + EXPECT_TRUE(table_func_op->set_child(mock_source_operator)); + + auto st = table_func_op->init(tplan_node_table_function_node, runtime_state.get()); + EXPECT_TRUE(st.ok()); + st = table_func_op->prepare(runtime_state.get()); + EXPECT_TRUE(st.ok()); + + st = table_func_op->setup_local_state(runtime_state.get(), local_state_info); + EXPECT_TRUE(st.ok()) << "setup_local_state failed: " << st.to_string(); + + local_state = runtime_state->get_local_state(table_func_op->operator_id()); + EXPECT_TRUE(local_state); + + st = local_state->open(runtime_state.get()); + EXPECT_TRUE(st.ok()) << "open failed: " << st.to_string(); + + return table_func_op; + } + + TTypeNode type_node_bool; + TTypeNode type_node_int; + TTypeNode type_node_arr; + TTypeDesc type_desc_int; + TTypeDesc type_desc_array_int; + TTypeDesc type_desc_boolean; + + ::doris::TSlotId slot_id = 0; + ::doris::TTupleId tuple_id = 0; + int32_t col_unique_id = 0; + + TTupleDescriptor tuple0; + TTupleDescriptor tuple1; + TTupleDescriptor tuple2; + + TSlotRef slot_ref_id; + TSlotRef slot_ref_tags; + TSlotRef slot_ref_unnest_tag; + + TSlotDescriptor slot_desc_id; + TSlotDescriptor slot_desc_tags; + TSlotDescriptor slot_desc_unnest_tag; + TSlotDescriptor slot_desc_id_2; + TSlotDescriptor slot_desc_unnest_tag2; + + std::unique_ptr runtime_state; + std::unique_ptr obj_pool; + std::shared_ptr query_ctx; + std::shared_ptr runtime_profile; +}; + +TEST_F(UnnestTest, inner) { + TDescriptorTable desc_table; + desc_table.tupleDescriptors.push_back(tuple0); + desc_table.tupleDescriptors.push_back(tuple1); + desc_table.tupleDescriptors.push_back(tuple2); + + desc_table.slotDescriptors.push_back(slot_desc_id); + desc_table.slotDescriptors.push_back(slot_desc_tags); + desc_table.slotDescriptors.push_back(slot_desc_unnest_tag); + desc_table.slotDescriptors.push_back(slot_desc_id_2); + desc_table.slotDescriptors.push_back(slot_desc_unnest_tag2); + + setup_exec_env(); + + // Set main TPlanNode properties + TPlanNode tplan_node_table_function_node; + setup_plan_node(tplan_node_table_function_node, false); + + DescriptorTbl* desc_tbl; + auto st = DescriptorTbl::create(obj_pool.get(), desc_table, &desc_tbl); + EXPECT_TRUE(st.ok()); + + vectorized::DataTypePtr data_type_int(std::make_shared()); + auto data_type_int_nullable = make_nullable(data_type_int); + vectorized::DataTypePtr data_type_array_type( + std::make_shared(data_type_int_nullable)); + auto data_type_array_type_nullable = make_nullable(data_type_array_type); + + auto build_input_block = [&](const std::vector& ids, + const std::vector>& array_rows) { + auto result_block = std::make_unique(); + auto id_column = ColumnInt32::create(); + for (const auto& id : ids) { + id_column->insert_data((const char*)(&id), 0); + } + result_block->insert(ColumnWithTypeAndName(make_nullable(std::move(id_column)), + data_type_int_nullable, "id")); + + auto arr_data_column = ColumnInt32::create(); + auto arr_offsets_column = ColumnOffset64::create(); + std::vector offsets; + offsets.push_back(0); + + for (const auto& array_row : array_rows) { + for (const auto& v : array_row) { + arr_data_column->insert_data((const char*)(&v), 0); + } + offsets.push_back(arr_data_column->size()); + } + for (size_t i = 1; i < offsets.size(); i++) { + arr_offsets_column->insert_data((const char*)(&offsets[i]), 0); + } + auto array_column = ColumnArray::create(make_nullable(std::move(arr_data_column)), + std::move(arr_offsets_column)); + result_block->insert(ColumnWithTypeAndName(make_nullable(std::move(array_column)), + data_type_array_type_nullable, "tags")); + return result_block; + }; + + { + std::shared_ptr table_func_op = + create_test_operators(desc_tbl, tplan_node_table_function_node); + auto* local_state = runtime_state->get_local_state(table_func_op->operator_id()); + auto* table_func_local_state = dynamic_cast(local_state); + std::vector ids = {1, 2}; + auto input_block = build_input_block(ids, {{1, 2, 1, 4, 5}, {1, 3, 3, 4, 5}}); + + table_func_local_state->_child_block = std::move(input_block); + table_func_local_state->_child_eos = true; + + auto expected_output_block = table_func_local_state->_child_block->clone_empty(); + auto unnested_tag_column = ColumnInt32::create(); + unnested_tag_column->insert_data((const char*)(ids.data()), 0); + unnested_tag_column->insert_data((const char*)(ids.data()), 0); + expected_output_block.insert(ColumnWithTypeAndName( + make_nullable(std::move(unnested_tag_column)), data_type_int_nullable, "tag")); + auto mutable_columns = expected_output_block.mutate_columns(); + mutable_columns[0]->insert_from( + *table_func_local_state->_child_block->get_by_position(0).column, 0); + mutable_columns[0]->insert_from( + *table_func_local_state->_child_block->get_by_position(0).column, 0); + mutable_columns[1]->insert_default(); + mutable_columns[1]->insert_default(); + expected_output_block.set_columns(std::move(mutable_columns)); + + // std::cout << "input block: \n" + // << table_func_local_state->_child_block->dump_data() << std::endl; + st = table_func_op->push(runtime_state.get(), table_func_local_state->_child_block.get(), + table_func_local_state->_child_eos); + ASSERT_TRUE(st.ok()) << "push failed: " << st.to_string(); + + bool eos = false; + Block output_block; + st = table_func_op->pull(runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + // std::cout << "output block: \n" << output_block.dump_data() << std::endl; + // std::cout << "expected output block: \n" << expected_output_block.dump_data() << std::endl; + EXPECT_FALSE(eos); + EXPECT_TRUE(ColumnHelper::block_equal(output_block, expected_output_block)); + + output_block.clear(); + st = table_func_op->pull(runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + // std::cout << "output block: \n" << output_block.dump_data() << std::endl; + EXPECT_TRUE(output_block.rows() == 0); + // EXPECT_TRUE(eos); + } +} + +TEST_F(UnnestTest, outer) { + TDescriptorTable desc_table; + desc_table.tupleDescriptors.push_back(tuple0); + desc_table.tupleDescriptors.push_back(tuple1); + desc_table.tupleDescriptors.push_back(tuple2); + + desc_table.slotDescriptors.push_back(slot_desc_id); + desc_table.slotDescriptors.push_back(slot_desc_tags); + desc_table.slotDescriptors.push_back(slot_desc_unnest_tag); + desc_table.slotDescriptors.push_back(slot_desc_id_2); + desc_table.slotDescriptors.push_back(slot_desc_unnest_tag2); + + setup_exec_env(); + + // Set main TPlanNode properties + TPlanNode tplan_node_table_function_node; + setup_plan_node(tplan_node_table_function_node, true); + + DescriptorTbl* desc_tbl; + auto st = DescriptorTbl::create(obj_pool.get(), desc_table, &desc_tbl); + EXPECT_TRUE(st.ok()); + + vectorized::DataTypePtr data_type_int(std::make_shared()); + auto data_type_int_nullable = make_nullable(data_type_int); + vectorized::DataTypePtr data_type_array_type( + std::make_shared(data_type_int_nullable)); + auto data_type_array_type_nullable = make_nullable(data_type_array_type); + + auto build_input_block = [&](const std::vector& ids, + const std::vector>& array_rows) { + auto result_block = std::make_unique(); + auto id_column = ColumnInt32::create(); + for (const auto& id : ids) { + id_column->insert_data((const char*)(&id), 0); + } + result_block->insert(ColumnWithTypeAndName(make_nullable(std::move(id_column)), + data_type_int_nullable, "id")); + + auto arr_data_column = ColumnInt32::create(); + auto arr_offsets_column = ColumnOffset64::create(); + std::vector offsets; + offsets.push_back(0); + + for (const auto& array_row : array_rows) { + for (const auto& v : array_row) { + arr_data_column->insert_data((const char*)(&v), 0); + } + offsets.push_back(arr_data_column->size()); + } + for (size_t i = 1; i < offsets.size(); i++) { + arr_offsets_column->insert_data((const char*)(&offsets[i]), 0); + } + auto array_column = ColumnArray::create(make_nullable(std::move(arr_data_column)), + std::move(arr_offsets_column)); + result_block->insert(ColumnWithTypeAndName(make_nullable(std::move(array_column)), + data_type_array_type_nullable, "tags")); + return result_block; + }; + + { + std::shared_ptr table_func_op = + create_test_operators(desc_tbl, tplan_node_table_function_node); + auto* local_state = runtime_state->get_local_state(table_func_op->operator_id()); + auto* table_func_local_state = dynamic_cast(local_state); + + std::vector ids = {1, 2}; + auto input_block = build_input_block(ids, {{1, 2, 1, 4, 5}, {1, 3, 3, 4, 5}}); + + table_func_local_state->_child_block = std::move(input_block); + table_func_local_state->_child_eos = true; + + auto expected_output_block = table_func_local_state->_child_block->clone_empty(); + auto unnested_tag_column = ColumnInt32::create(); + unnested_tag_column->insert_data((const char*)(ids.data()), 0); + unnested_tag_column->insert_data((const char*)(ids.data()), 0); + expected_output_block.insert(ColumnWithTypeAndName( + make_nullable(std::move(unnested_tag_column)), data_type_int_nullable, "tag")); + auto mutable_columns = expected_output_block.mutate_columns(); + mutable_columns[0]->insert_from( + *table_func_local_state->_child_block->get_by_position(0).column, 0); + mutable_columns[0]->insert_from( + *table_func_local_state->_child_block->get_by_position(0).column, 0); + mutable_columns[1]->insert_default(); + mutable_columns[1]->insert_default(); + expected_output_block.set_columns(std::move(mutable_columns)); + + // std::cout << "input block: \n" + // << table_func_local_state->_child_block->dump_data() << std::endl; + st = table_func_op->push(runtime_state.get(), table_func_local_state->_child_block.get(), + table_func_local_state->_child_eos); + ASSERT_TRUE(st.ok()) << "push failed: " << st.to_string(); + + bool eos = false; + Block output_block; + st = table_func_op->pull(runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + // std::cout << "output block: \n" << output_block.dump_data() << std::endl; + // std::cout << "expected output block: \n" << expected_output_block.dump_data() << std::endl; + EXPECT_FALSE(eos); + EXPECT_TRUE(ColumnHelper::block_equal(output_block, expected_output_block)); + + output_block.clear(); + expected_output_block.clear_column_data(); + mutable_columns = expected_output_block.mutate_columns(); + mutable_columns[0]->insert_from( + *table_func_local_state->_child_block->get_by_position(0).column, 1); + mutable_columns[1]->insert_default(); + mutable_columns[2]->insert_default(); + expected_output_block.set_columns(std::move(mutable_columns)); + st = table_func_op->pull(runtime_state.get(), &output_block, &eos); + ASSERT_TRUE(st.ok()) << "pull failed: " << st.to_string(); + // std::cout << "output block: \n" << output_block.dump_data() << std::endl; + // std::cout << "expected output block: \n" << expected_output_block.dump_data() << std::endl; + EXPECT_TRUE(eos); + EXPECT_TRUE(ColumnHelper::block_equal(output_block, expected_output_block)); + } +} } // namespace doris::pipeline \ No newline at end of file diff --git a/regression-test/data/nereids_function_p0/gen_function/unnest.out b/regression-test/data/nereids_function_p0/gen_function/unnest.out index 4dfe4464a93482..b24d6f974d72ca 100644 --- a/regression-test/data/nereids_function_p0/gen_function/unnest.out +++ b/regression-test/data/nereids_function_p0/gen_function/unnest.out @@ -588,6 +588,30 @@ Shirt [4] 4 4 3 Basketball ["Sports", "Outdoor"] Sports 4 Badminton Racket ["Sports", "Equipment"] Sports +-- !sql_inner_join3 -- +1 +5 + +-- !sql_inner_join4 -- +Laptop +Shirt + +-- !sql_inner_join5 -- +["Clothing", "Office", "Shirt"] +["Electronics", "Office", "High-End", "Laptop"] + +-- !sql_inner_join6 -- +Laptop +Shirt + +-- !sql_inner_join7 -- +1 Laptop +5 Shirt + +-- !sql_inner_join8 -- +1 +1 + -- !sql_left_join0 -- 1 Laptop ["Electronics", "Office", "High-End", "Laptop"] Laptop 2 Mechanical Keyboard ["Electronics", "Accessories"] \N @@ -613,6 +637,48 @@ Shirt [4] 4 4 4 Badminton Racket ["Sports", "Equipment"] Sports 5 Shirt ["Clothing", "Office", "Shirt"] \N +-- !sql_left_join3 -- +1 +2 +3 +4 +5 + +-- !sql_left_join4 -- +Badminton Racket +Basketball +Laptop +Mechanical Keyboard +Shirt + +-- !sql_left_join5 -- +["Clothing", "Office", "Shirt"] +["Electronics", "Accessories"] +["Electronics", "Office", "High-End", "Laptop"] +["Sports", "Equipment"] +["Sports", "Outdoor"] + +-- !sql_left_join6 -- +\N +\N +\N +Laptop +Shirt + +-- !sql_left_join7 -- +1 Laptop +2 \N +3 \N +4 \N +5 Shirt + +-- !sql_left_join8 -- +1 +1 +1 +1 +1 + -- !big_array_unnest_all -- 1 10 [1, 1, 2, 4, 5, 6, 7, 8, 9, 10] 2 10 [1, 3, 3, 4, 5, 6, 7, 8, 9, 10] diff --git a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy index 7632c066fb1f38..de2296eae75f18 100644 --- a/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy +++ b/regression-test/suites/nereids_function_p0/gen_function/unnest.groovy @@ -384,6 +384,49 @@ suite("nereids_unnest_fn") { items_dict_unnest_t INNER JOIN lateral unnest(tags) AS t(tag) ON tag in ('Electronics', 'Sports'); ''' + order_qt_sql_inner_join3 ''' + SELECT + id + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join4 ''' + SELECT + name + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join5 ''' + SELECT + tags + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join6 ''' + SELECT + t.tag + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join7 ''' + SELECT + id, + t.tag + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_inner_join8 ''' + SELECT + 1 + FROM + items_dict_unnest_t + INNER JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' order_qt_sql_left_join0 ''' SELECT @@ -416,6 +459,50 @@ suite("nereids_unnest_fn") { LEFT JOIN lateral unnest(tags) AS t(tag) ON tag in ('Electronics', 'Sports'); ''' + order_qt_sql_left_join3 ''' + SELECT + id + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_left_join4 ''' + SELECT + name + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_left_join5 ''' + SELECT + tags + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_left_join6 ''' + SELECT + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_left_join7 ''' + SELECT + id, + t.tag + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + order_qt_sql_left_join8 ''' + SELECT + 1 + FROM + items_dict_unnest_t + LEFT JOIN lateral unnest(tags) AS t(tag) ON t.tag = name; + ''' + test { sql ''' SELECT From 90823fc1ca7a80baaa44bbb60930bb6a701c5012 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Mon, 22 Dec 2025 15:37:06 +0800 Subject: [PATCH 10/14] improve --- .../pipeline/exec/table_function_operator.cpp | 40 +++++++++++-------- .../pipeline/exec/table_function_operator.h | 2 +- 2 files changed, 24 insertions(+), 18 deletions(-) diff --git a/be/src/pipeline/exec/table_function_operator.cpp b/be/src/pipeline/exec/table_function_operator.cpp index ea400a9ee45ea2..b746fedb89896c 100644 --- a/be/src/pipeline/exec/table_function_operator.cpp +++ b/be/src/pipeline/exec/table_function_operator.cpp @@ -22,6 +22,7 @@ #include "pipeline/exec/operator.h" #include "util/simd/bits.h" +#include "vec/common/custom_allocator.h" #include "vec/core/block.h" #include "vec/core/column_numbers.h" #include "vec/exprs/table_function/table_function_factory.h" @@ -254,8 +255,8 @@ Status TableFunctionLocalState::_get_expanded_block_for_outer_conjuncts( } } - std::vector child_row_to_output_rows_indices; - std::vector handled_row_indices; + DorisVector child_row_to_output_rows_indices; + DorisVector handled_row_indices; bool child_block_empty = _child_block->empty(); if (!child_block_empty) { child_row_to_output_rows_indices.push_back(0); @@ -404,25 +405,30 @@ Status TableFunctionLocalState::_get_expanded_block_for_outer_conjuncts( if (-1 == _cur_child_offset) { // Finished handling current child block, - vectorized::MutableBlock m_block2 = - vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, - p._output_slots); - vectorized::MutableColumns& columns2 = m_block2.mutable_columns(); auto child_block_row_count = _child_block->rows(); - for (size_t i = 0; i != child_block_row_count; i++) { + DorisVector null_row_indices; + for (uint32_t i = 0; i != child_block_row_count; i++) { if (!_child_rows_has_output[i]) { - // insert one row with NULL for this child row - for (auto index : p._output_slot_indexs) { - auto src_column = _child_block->get_by_position(index).column; - columns2[index]->insert_from(*src_column, i); - } - for (auto index : p._useless_slot_indexs) { - columns2[index]->insert_default(); - } - columns2[child_slot_count]->insert_default(); + null_row_indices.push_back(i); } } - output_block->set_columns(std::move(columns2)); + if (!null_row_indices.empty()) { + vectorized::MutableBlock m_block2 = + vectorized::VectorizedUtils::build_mutable_mem_reuse_block(output_block, + p._output_slots); + vectorized::MutableColumns& columns2 = m_block2.mutable_columns(); + for (auto index : p._output_slot_indexs) { + auto src_column = _child_block->get_by_position(index).column; + columns2[index]->insert_indices_from( + *src_column, null_row_indices.data(), + null_row_indices.data() + null_row_indices.size()); + } + for (auto index : p._useless_slot_indexs) { + columns2[index]->insert_many_defaults(null_row_indices.size()); + } + columns2[child_slot_count]->insert_many_defaults(null_row_indices.size()); + output_block->set_columns(std::move(columns2)); + } _child_rows_has_output.clear(); _child_block->clear_column_data(_parent->cast() ._child->row_desc() diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h index fdbde3b46fd9c9..56f9f116ec8270 100644 --- a/be/src/pipeline/exec/table_function_operator.h +++ b/be/src/pipeline/exec/table_function_operator.h @@ -78,7 +78,7 @@ class TableFunctionLocalState MOCK_REMOVE(final) : public PipelineXLocalState<> std::unique_ptr _child_block; int _current_row_insert_times = 0; bool _child_eos = false; - std::vector _child_rows_has_output; + DorisVector _child_rows_has_output; RuntimeProfile::Counter* _init_function_timer = nullptr; RuntimeProfile::Counter* _process_rows_timer = nullptr; From ab7e2f86e287071f46b6a4f23aa3c891a9e51a4b Mon Sep 17 00:00:00 2001 From: jacktengg Date: Wed, 24 Dec 2025 11:54:32 +0800 Subject: [PATCH 11/14] debug: print plan at BE to debug failure p0 case --- regression-test/pipeline/p0/conf/be.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/pipeline/p0/conf/be.conf b/regression-test/pipeline/p0/conf/be.conf index 01df493f002f37..fd11c21f55fd0a 100644 --- a/regression-test/pipeline/p0/conf/be.conf +++ b/regression-test/pipeline/p0/conf/be.conf @@ -35,7 +35,7 @@ JEMALLOC_PROF_PRFIX="jemalloc_heap_profile_" # INFO, WARNING, ERROR, FATAL sys_log_level = INFO -# sys_log_verbose_modules=dictionary_factory +sys_log_verbose_modules=fragment_mgr be_port = 9161 webserver_port = 8141 heartbeat_service_port = 9151 From 19b255c4a92d7b68c55418564a86f7ba25892791 Mon Sep 17 00:00:00 2001 From: lichi Date: Fri, 26 Dec 2025 10:31:41 +0800 Subject: [PATCH 12/14] add cases --- .../nereids/parser/LogicalPlanBuilder.java | 54 ++++++-- .../rules/analysis/BindExpression.java | 35 +++++- .../nereids/rules/analysis/CheckAnalysis.java | 3 + .../rewrite/PushDownUnnestInProject.java | 5 +- .../functions/generator/Unnest.java | 14 +-- .../trees/plans/logical/LogicalGenerate.java | 7 +- .../plans/physical/PhysicalGenerate.java | 2 +- .../doris/nereids/util/ExpressionUtils.java | 87 +++++++------ .../doris/planner/TableFunctionNode.java | 4 +- .../nereids/parser/NereidsParserTest.java | 7 ++ .../analysis/NormalizeAggregateTest.java | 27 ++++ .../rules/rewrite/MergeGeneratesTest.java | 118 ++++++++++++++++++ .../nereids/util/ExpressionUtilsTest.java | 43 +++++++ 13 files changed, 334 insertions(+), 72 deletions(-) create mode 100644 fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeGeneratesTest.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 556f45084dceda..a5c35f2d54a966 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1116,6 +1116,9 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor { private static String JOB_NAME = "jobName"; private static String TASK_ID = "taskId"; + private static String DEFAULT_NESTED_COLUMN_NAME = "unnest"; + private static String DEFAULT_ORDINALITY_COLUMN_NAME = "ordinality"; + // Sort the parameters with token position to keep the order with original placeholders // in prepared statement.Otherwise, the order maybe broken private final Map tokenPosToParameters = Maps.newTreeMap((pos1, pos2) -> { @@ -2726,15 +2729,13 @@ public LogicalPlan visitUnnestFunction(DorisParser.UnnestFunctionContext ctx) { } private LogicalPlan withUnnest(DorisParser.UnnestContext ctx) { - String defaultNestedColumnName = "unnest"; - String defaultOrdinalityColumnName = "ordinality"; List arguments = ctx.expression().stream() .map(this::typedVisit) .collect(ImmutableList.toImmutableList()); boolean needOrdinality = ctx.ORDINALITY() != null; int size = arguments.size(); - String generateName = ctx.tableName != null ? ctx.tableName.getText() : defaultNestedColumnName; + String generateName = ctx.tableName != null ? ctx.tableName.getText() : DEFAULT_NESTED_COLUMN_NAME; // do same thing as later view explode map type, we need to add a project to convert map to struct int argumentsSize = size + (needOrdinality ? 1 : 0); List nestedColumnNames = new ArrayList<>(argumentsSize); @@ -2744,25 +2745,25 @@ private LogicalPlan withUnnest(DorisParser.UnnestContext ctx) { nestedColumnNames.add(ctx.columnNames.get(i).getText()); } for (int i = 0; i < size - columnNamesSize; ++i) { - nestedColumnNames.add(defaultNestedColumnName); + nestedColumnNames.add(DEFAULT_NESTED_COLUMN_NAME); } if (needOrdinality && columnNamesSize < argumentsSize) { - nestedColumnNames.add(defaultOrdinalityColumnName); + nestedColumnNames.add(DEFAULT_ORDINALITY_COLUMN_NAME); } } else { if (size == 1) { nestedColumnNames.add(generateName); } else { for (int i = 0; i < size; ++i) { - nestedColumnNames.add(defaultNestedColumnName); + nestedColumnNames.add(DEFAULT_NESTED_COLUMN_NAME); } } if (needOrdinality) { - nestedColumnNames.add(defaultOrdinalityColumnName); + nestedColumnNames.add(DEFAULT_ORDINALITY_COLUMN_NAME); } } String columnName = nestedColumnNames.get(0); - Unnest unnest = new Unnest(false, needOrdinality, arguments); + Unnest unnest = new Unnest(arguments, needOrdinality, false); // only unnest use LogicalOneRowRelation as LogicalGenerate's child, // so we can check LogicalGenerate's child to know if it's unnest function return new LogicalGenerate<>(ImmutableList.of(unnest), @@ -4413,8 +4414,23 @@ private LogicalPlan withJoinRelations(LogicalPlan input, RelationContext ctx) { if (ids == null) { LogicalPlan right = plan(join.relationPrimary()); if (right instanceof LogicalGenerate - && right.child(0) instanceof LogicalOneRowRelation && ((LogicalGenerate) right).getGenerators().get(0) instanceof Unnest) { + /* + SELECT + id, + scores, + unnest + FROM + student_unnest_t + LEFT JOIN unnest(scores) AS unnest ON TRUE + + above sql will be converted to: + UnboundResultSink[6] + +--LogicalProject[5] ( distinct=false, projects=['id, 'scores, 'unnest] ) + +--LogicalGenerate ( generators=[unnest('scores)], generatorOutput=['unnest.unnest],... + +--LogicalCheckPolicy + +--UnboundRelation ( id=RelationId#0, nameParts=student_unnest_t ) + */ if (joinType.isLeftJoin() || joinType.isInnerJoin() || joinType.isCrossJoin()) { LogicalGenerate oldRight = (LogicalGenerate) right; Unnest oldGenerator = (Unnest) oldRight.getGenerators().get(0); @@ -4656,7 +4672,25 @@ private LogicalPlan withRelations(LogicalPlan inputPlan, List r for (RelationContext relation : relations) { // build left deep join tree LogicalPlan right = withJoinRelations(visitRelation(relation), relation); - // check if it's unnest + /* + SELECT + items_dict_unnest_t.id, + items_dict_unnest_t.name, + t.tag, + t.ord + FROM + items_dict_unnest_t, + unnest(items_dict_unnest_t.tags) AS t(tag, ord) + + there is unnest in from clause, we should not generate plan like items_dict_unnest_t join unnest... + instead, we should put items_dict_unnest_t as LogicalGenerate's child like bellow: + + UnboundResultSink[6] + +--LogicalProject[5] ( distinct=false, projects=['items_dict_unnest_t.id, 'items_dict_unnest_t.name,.. ) + +--LogicalGenerate ( generators=[unnest('items_dict_unnest_t.tags)], generatorOutput=['t.tag],... ) + +--LogicalCheckPolicy + +--UnboundRelation ( id=RelationId#0, nameParts=items_dict_unnest_t ) + */ boolean shouldBeParent = right instanceof LogicalGenerate && right.child(0) instanceof LogicalOneRowRelation; left = (left == null) ? right diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 554661be296b3d..b040103f1b814a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -57,6 +57,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue; import org.apache.doris.nereids.trees.expressions.functions.agg.NullableAggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.scalar.GroupingScalarFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.StructElement; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; @@ -94,6 +95,7 @@ import org.apache.doris.nereids.trees.plans.logical.ProjectProcessor; import org.apache.doris.nereids.trees.plans.visitor.InferPlanOutputAlias; import org.apache.doris.nereids.types.BooleanType; +import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.StructField; import org.apache.doris.nereids.types.StructType; import org.apache.doris.nereids.util.ExpressionUtils; @@ -283,7 +285,29 @@ private LogicalPlan bindGenerate(MatchingContext> ctx) { if (!(boundGenerator instanceof TableGeneratingFunction)) { throw new AnalysisException(boundGenerator.toSql() + " is not a TableGeneratingFunction"); } - Function generator = ExpressionUtils.convertUnnest((Function) boundGenerator); + Function generator = (Function) boundGenerator; + if (boundGenerator instanceof Unnest) { + Unnest unnest = (Unnest) boundGenerator; + DataType dataType = unnest.child(0).getDataType(); + int columnNamesSize = generate.getExpandColumnAlias().get(0).size(); + int argumentsSize = boundGenerator.getArguments().size() + (unnest.needOrdinality() ? 1 : 0); + String generateName = generate.getGeneratorOutput().get(0).getQualifier().get(0); + if (dataType.isArrayType() || dataType.isBitmapType()) { + if (columnNamesSize > argumentsSize) { + throw new AnalysisException( + String.format("table %s has %d columns available but %d columns specified", + generateName, argumentsSize, columnNamesSize)); + } + } else if (dataType.isMapType()) { + if (columnNamesSize > 2) { + throw new AnalysisException( + String.format("table %s has 2 columns available but %d columns specified", + generateName, columnNamesSize)); + } + } + + generator = ExpressionUtils.convertUnnest(unnest); + } boundGenerators.add(generator); Slot boundSlot = new SlotReference(slot.getNameParts().get(1), generator.getDataType(), @@ -324,7 +348,7 @@ boundSlot, new StringLiteral(fields.get(idx).getName())), * */ int conjunctSize = generate.getConjuncts().size(); - List newConjuncts = new ArrayList<>(conjunctSize); + ImmutableList.Builder newConjuncts = new Builder(); if (conjunctSize > 0) { List childOutputs = generate.child().getOutput(); List conjunctsScopeSlots = new ArrayList<>(expandAlias.size() + childOutputs.size()); @@ -346,16 +370,15 @@ boundSlot, new StringLiteral(fields.get(idx).getName())), } LogicalGenerate logicalGenerate = new LogicalGenerate<>( - boundGenerators.build(), outputSlots.build(), ImmutableList.of(), newConjuncts, generate.child()); + boundGenerators.build(), outputSlots.build(), ImmutableList.of(), newConjuncts.build(), + generate.child()); if (!expandAlias.isEmpty()) { // project should contain: generator.child slot + expandAlias List allProjectSlots = new ArrayList<>(generate.child().getOutput().size() + expandAlias.size()); if (!(generate.child() instanceof LogicalOneRowRelation)) { // project should contain: generator.child slot + expandAlias except: - allProjectSlots.addAll(generate.child().getOutput().stream() - .map(NamedExpression.class::cast) - .collect(Collectors.toList())); + allProjectSlots.addAll(new ArrayList<>(generate.child().getOutput())); } else { // unnest with literal argument as unnest([1,2,3]) // we should not add LogicalOneRowRelation's output slot in this case diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java index 44c9522d902f42..19c45e6105247b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java @@ -161,6 +161,9 @@ private void checkUnexpectedExpressionTypes(Plan plan, Class { for (Class type : unexpectedExpressionTypes) { + // PushDownUnnestInProject will push down Unnest in Project list in rewrite phase + // it relays on many rules like normalizeXXX to separate Unnest into LogicalProject first + // here, we allow Unnest in analysis phase and deal with it in rewrite phase if (type.isInstance(e) && !(e instanceof Unnest)) { throw new AnalysisException(plan.getType() + " can not contains " + type.getSimpleName() + " expression: " + ((Expression) e).toSql()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java index 1708e23332c52b..ebe0ba7b4c24a4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownUnnestInProject.java @@ -116,6 +116,9 @@ private Unnest validateAndMergeUnnest(List functions) { } Expression expr = unnest.child(0); DataType dataType = expr.getDataType(); + // we only care about generic type info are same + // so ARRAY, ARRAY will be treated as same type + // we use 1, 2, 3 as some mark value for array, map and bitmap type separately if (dataType.isArrayType()) { typeCounter += 1; } else if (dataType.isMapType()) { @@ -130,7 +133,7 @@ private Unnest validateAndMergeUnnest(List functions) { throw new AnalysisException("multiple UNNEST functions in same place must have ARRAY argument type"); } if (typeCounter == 1 * size || typeCounter == 2 * size || typeCounter == 3 * size) { - return new Unnest(false, false, expressions); + return new Unnest(expressions, false, false); } else { throw new AnalysisException("UNNEST functions must have same argument type"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java index 61ac628640908a..897052ffd884f0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/generator/Unnest.java @@ -53,23 +53,23 @@ public class Unnest extends TableGeneratingFunction implements CustomSignature, * constructor with one argument. */ public Unnest(Expression argument) { - this(false, false, ImmutableList.of(argument)); + this(ImmutableList.of(argument), false, false); } /** * constructor with more argument. */ - public Unnest(boolean isOuter, boolean needOrdinality, List arguments) { + public Unnest(List arguments, boolean needOrdinality, boolean isOuter) { super("unnest", arguments); - this.isOuter = isOuter; this.needOrdinality = needOrdinality; + this.isOuter = isOuter; } /** constructor for withChildren and reuse signature */ - private Unnest(boolean isOuter, boolean needOrdinality, GeneratorFunctionParams functionParams) { + private Unnest(GeneratorFunctionParams functionParams, boolean needOrdinality, boolean isOuter) { super(functionParams); - this.isOuter = isOuter; this.needOrdinality = needOrdinality; + this.isOuter = isOuter; } /** @@ -78,11 +78,11 @@ private Unnest(boolean isOuter, boolean needOrdinality, GeneratorFunctionParams @Override public Unnest withChildren(List children) { Preconditions.checkArgument(!children.isEmpty()); - return new Unnest(isOuter, needOrdinality, getFunctionParams(children)); + return new Unnest(getFunctionParams(children), needOrdinality, isOuter); } public Unnest withOuter(boolean isOuter) { - return new Unnest(isOuter, needOrdinality, getFunctionParams(children)); + return new Unnest(getFunctionParams(children), needOrdinality, isOuter); } public boolean isOuter() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java index b38830bdaf94d2..d5adde59537522 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java @@ -35,7 +35,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.util.HashSet; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -119,7 +121,10 @@ public List getExpressions() { @Override public Set getInputSlots() { - return PlanUtils.fastGetInputSlots(generators); + Set slots = new HashSet<>(); + slots.addAll(PlanUtils.fastGetInputSlots(generators)); + slots.addAll(PlanUtils.fastGetInputSlots(conjuncts)); + return Sets.difference(slots, Sets.newHashSet(generatorOutput)); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java index b84a264bf45211..829ec03588d3c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalGenerate.java @@ -66,7 +66,7 @@ public PhysicalGenerate(List generators, List generatorOutput, this.generators = ImmutableList.copyOf(Objects.requireNonNull(generators, "predicates can not be null")); this.generatorOutput = ImmutableList.copyOf(Objects.requireNonNull(generatorOutput, "generatorOutput can not be null")); - this.conjuncts = ImmutableList.copyOf(conjuncts); + this.conjuncts = ImmutableList.copyOf(Objects.requireNonNull(conjuncts)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 830fd4772a9828..7a7d12626be4d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -606,8 +606,8 @@ public static List replaceNamedExpressions(List e instanceof UniqueFunction ? ((UniqueFunction) e).withIgnoreUniqueId(ignoreUniqueId) : e); + return expression.rewriteDownShortCircuit(e -> + e instanceof UniqueFunction ? ((UniqueFunction) e).withIgnoreUniqueId(ignoreUniqueId) : e); } public static List rewriteDownShortCircuit( @@ -682,9 +682,10 @@ public static boolean canInferNotNullForMarkSlot(Expression predicate, Expressio * the mark slot can be non-nullable boolean * and in semi join, we can safely change the mark conjunct to hash conjunct */ - ImmutableList literals = ImmutableList.of(NullLiteral.BOOLEAN_INSTANCE, BooleanLiteral.FALSE); - List markJoinSlotReferenceList = new ArrayList<>( - (predicate.collect(MarkJoinSlotReference.class::isInstance))); + ImmutableList literals = + ImmutableList.of(NullLiteral.BOOLEAN_INSTANCE, BooleanLiteral.FALSE); + List markJoinSlotReferenceList = + new ArrayList<>((predicate.collect(MarkJoinSlotReference.class::isInstance))); int markSlotSize = markJoinSlotReferenceList.size(); int maxMarkSlotCount = 4; // if the conjunct has mark slot, and maximum 4 mark slots(for performance) @@ -711,7 +712,8 @@ public static boolean canInferNotNullForMarkSlot(Expression predicate, Expressio } Expression evalResult = FoldConstantRule.evaluate( ExpressionUtils.replace(predicate, replaceMap), - ctx); + ctx + ); if (evalResult.equals(BooleanLiteral.TRUE)) { if (meetNullOrFalse) { @@ -750,7 +752,8 @@ public static Set inferNotNullSlots(Set predicates, CascadesCo replaceMap.put(slot, nullLiteral); Expression evalExpr = FoldConstantRule.evaluate( ExpressionUtils.replace(predicate, replaceMap), - new ExpressionRewriteContext(cascadesContext)); + new ExpressionRewriteContext(cascadesContext) + ); if (evalExpr.isNullLiteral() || BooleanLiteral.FALSE.equals(evalExpr)) { notNullSlots.add(slot); } @@ -944,8 +947,7 @@ public static Set mutableCollect(List expressions, public static List collectAll(Collection expressions, Predicate> predicate) { switch (expressions.size()) { - case 0: - return ImmutableList.of(); + case 0: return ImmutableList.of(); default: { ImmutableList.Builder result = ImmutableList.builder(); for (Expression expr : expressions) { @@ -1054,13 +1056,14 @@ public static Expression getSingleNumericSlotOrExpressionCoveredByCast(Expressio */ public static boolean checkSlotConstant(Slot slot, Set predicates) { return predicates.stream().anyMatch(predicate -> { - if (predicate instanceof EqualTo) { - EqualTo equalTo = (EqualTo) predicate; - return (equalTo.left() instanceof Literal && equalTo.right().equals(slot)) - || (equalTo.right() instanceof Literal && equalTo.left().equals(slot)); - } - return false; - }); + if (predicate instanceof EqualTo) { + EqualTo equalTo = (EqualTo) predicate; + return (equalTo.left() instanceof Literal && equalTo.right().equals(slot)) + || (equalTo.right() instanceof Literal && equalTo.left().equals(slot)); + } + return false; + } + ); } /** @@ -1178,7 +1181,8 @@ public static Literal analyzeAndFoldToLiteral(ConnectContext ctx, Expression exp } ExpressionRewriteContext context = new ExpressionRewriteContext(cascadesContext); ExpressionRuleExecutor executor = new ExpressionRuleExecutor(ImmutableList.of( - ExpressionRewrite.bottomUp(ReplaceVariableByLiteral.INSTANCE))); + ExpressionRewrite.bottomUp(ReplaceVariableByLiteral.INSTANCE) + )); Expression rewrittenExpression = executor.rewrite(analyzedExpr, context); Expression foldExpression = FoldConstantRule.evaluate(rewrittenExpression, context); if (foldExpression instanceof Literal) { @@ -1253,8 +1257,8 @@ public static boolean containsCaseWhenLikeType(Expression expression) { public static Optional> getCaseWhenLikeBranchResults(Expression expression) { if (expression instanceof CaseWhen) { CaseWhen caseWhen = (CaseWhen) expression; - ImmutableList.Builder builder = ImmutableList - .builderWithExpectedSize(caseWhen.getWhenClauses().size() + 1); + ImmutableList.Builder builder + = ImmutableList.builderWithExpectedSize(caseWhen.getWhenClauses().size() + 1); for (WhenClause whenClause : caseWhen.getWhenClauses()) { builder.add(whenClause.getResult()); } @@ -1309,34 +1313,29 @@ public static boolean containUniqueFunctionExistMultiple(Collection args = unnest.getArguments(); - if (dataType.isArrayType()) { - Expression[] arrayArgs = args.toArray(new Expression[0]); - return unnest.isOuter() - ? unnest.needOrdinality() ? new PosExplodeOuter(arrayArgs) : new ExplodeOuter(arrayArgs) - : unnest.needOrdinality() ? new PosExplode(arrayArgs) : new Explode(arrayArgs); + public static org.apache.doris.nereids.trees.expressions.functions.Function convertUnnest(Unnest unnest) { + DataType dataType = unnest.child(0).getDataType(); + List args = unnest.getArguments(); + if (dataType.isArrayType()) { + Expression[] arrayArgs = args.toArray(new Expression[0]); + return unnest.isOuter() + ? unnest.needOrdinality() ? new PosExplodeOuter(arrayArgs) : new ExplodeOuter(arrayArgs) + : unnest.needOrdinality() ? new PosExplode(arrayArgs) : new Explode(arrayArgs); + } else { + if (unnest.needOrdinality()) { + throw new AnalysisException(String.format("only ARRAY support WITH ORDINALITY," + + " but argument's type is %s", dataType)); + } + if (dataType.isMapType()) { + return unnest.isOuter() ? new ExplodeMapOuter(args.get(0)) : new ExplodeMap(args.get(0)); + } else if (dataType.isBitmapType()) { + return unnest.isOuter() ? new ExplodeBitmapOuter(args.get(0)) : new ExplodeBitmap(args.get(0)); } else { - if (unnest.needOrdinality()) { - throw new AnalysisException(String.format("only ARRAY support WITH ORDINALITY," - + " but argument's type is %s", dataType)); - } - if (dataType.isMapType()) { - return unnest.isOuter() ? new ExplodeMapOuter(args.get(0)) : new ExplodeMap(args.get(0)); - } else if (dataType.isBitmapType()) { - return unnest.isOuter() ? new ExplodeBitmapOuter(args.get(0)) : new ExplodeBitmap(args.get(0)); - } else { - throw new AnalysisException(String.format("UNNEST function doesn't support %s argument type, " - + "please try to use lateral view and explode_* function set instead", dataType.toSql())); - } + throw new AnalysisException(String.format("UNNEST function doesn't support %s argument type, " + + "please try to use lateral view and explode_* function set instead", dataType.toSql())); } - } else { - return function; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java index 5195c1f3a06db7..21babbcc98942c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/TableFunctionNode.java @@ -37,7 +37,7 @@ public class TableFunctionNode extends PlanNode { // The output slot ids of TableFunctionNode // Only the slot whose id is in this list will be output by TableFunctionNode private List outputSlotIds = Lists.newArrayList(); - private List expandConjuncts = Lists.newArrayList(); + private List expandConjuncts; public TableFunctionNode(PlanNodeId id, PlanNode inputNode, TupleId lateralViewTupleId, ArrayList fnCallExprList, List outputSlotIds, List expandConjuncts) { @@ -57,7 +57,7 @@ public TableFunctionNode(PlanNodeId id, PlanNode inputNode, TupleId lateralViewT this.fnCallExprList = fnCallExprList; this.outputSlotIds = outputSlotIds; this.children.add(inputNode); - this.expandConjuncts.addAll(expandConjuncts); + this.expandConjuncts = expandConjuncts; } public void setOutputSlotIds(List outputSlotIds) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index 6b134440ae61df..4ba56765a1daee 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.literal.DecimalLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLikeLiteral; @@ -1499,4 +1500,10 @@ public void testMergeInto() throws Exception { + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; Assertions.assertThrows(ParseException.class, () -> parser.parseSingle(invalidSql4)); } + + @Test + public void testUnnest() { + String sql = "SELECT t.* FROM LATERAL unnest([1,2], ['hi','hello']) WITH ORDINALITY AS t(c1,c2);"; + parsePlan(sql).matches(logicalGenerate().when(plan -> plan.getGenerators().get(0) instanceof Unnest)); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java index 6a66c09b445785..623593c1491bed 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/NormalizeAggregateTest.java @@ -28,6 +28,8 @@ import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; +import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -35,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.FieldChecker; import org.apache.doris.nereids.util.LogicalPlanBuilder; import org.apache.doris.nereids.util.MemoPatternMatchSupported; @@ -689,6 +692,30 @@ void testAggFunctionNullabe2() { ); } + @Test + public void testUnnestFunction() { + NamedExpression key = rStudent.getOutput().get(2).toSlot(); + List arguments = Lists.newArrayList( + new ArrayLiteral(Lists.newArrayList(new IntegerLiteral(1)))); + NamedExpression aggregateFunction = new Alias(new Sum(new Unnest(arguments, false, false)), "sum"); + List groupExpressionList = Lists.newArrayList(key); + List outputExpressionList = Lists.newArrayList(key, aggregateFunction); + Plan root = new LogicalAggregate<>(groupExpressionList, outputExpressionList, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new NormalizeAggregate()) + .matchesFromRoot( + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan() + ).when(project -> ExpressionUtils.containsTypes( + project.getProjects(), Unnest.class)) + ) + ) + ); + } + private void checkExprsToSql(Collection expressions, String... exprsToSql) { Assertions.assertEquals(Arrays.asList(exprsToSql), expressions.stream().map(Expression::toSql).collect(Collectors.toList())); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeGeneratesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeGeneratesTest.java new file mode 100644 index 00000000000000..f50d9c4400c7c6 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeGeneratesTest.java @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.util.MemoPatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.nereids.util.PlanConstructor; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +/** + * Tests for {@link MergeGenerates}. + */ +class MergeGeneratesTest implements MemoPatternMatchSupported { + + private final LogicalOlapScan scan = PlanConstructor.newLogicalOlapScan(0, "t1", 0); + + @Test + void testMergeGenerates() { + // bottom generate + SlotReference bottomOut = new SlotReference("g1", IntegerType.INSTANCE); + Unnest bottomGen = new Unnest(new IntegerLiteral(1)); + LogicalGenerate bottom = new LogicalGenerate<>( + ImmutableList.of(bottomGen), ImmutableList.of(bottomOut), scan); + + // top generate (does not reference bottom output and has empty conjuncts) + SlotReference topOut = new SlotReference("g2", IntegerType.INSTANCE); + Unnest topGen = new Unnest(new IntegerLiteral(2)); + LogicalGenerate top = new LogicalGenerate<>( + ImmutableList.of(topGen), ImmutableList.of(topOut), bottom); + + PlanChecker.from(new ConnectContext(), top).applyBottomUp(new MergeGenerates()) + .matches(logicalGenerate().when(g -> g.getGenerators().size() == 2 + && g.getGeneratorOutput().size() == 2 + && !(g.child(0) instanceof LogicalGenerate))); + } + + @Test + void testDoNotMergeWhenTopHasConjuncts() { + SlotReference bottomOut = new SlotReference("g1", IntegerType.INSTANCE); + Unnest bottomGen = new Unnest(new IntegerLiteral(1)); + LogicalGenerate bottom = new LogicalGenerate<>( + ImmutableList.of(bottomGen), ImmutableList.of(bottomOut), scan); + + // top has a non-empty conjuncts list -> should NOT merge + SlotReference topOut = new SlotReference("g2", IntegerType.INSTANCE); + Unnest topGen = new Unnest(new IntegerLiteral(2)); + LogicalGenerate top = new LogicalGenerate<>( + ImmutableList.of(topGen), ImmutableList.of(topOut), ImmutableList.of(ImmutableList.of("xx")), + ImmutableList.of(BooleanLiteral.TRUE), bottom); + + PlanChecker.from(new ConnectContext(), top).applyBottomUp(new MergeGenerates()) + .printlnAllTree() + .matches(logicalGenerate(logicalGenerate())); + } + + @Test + void testDoNotMergeWhenBottomHasConjuncts() { + SlotReference bottomOut = new SlotReference("g1", IntegerType.INSTANCE); + Unnest bottomGen = new Unnest(new IntegerLiteral(1)); + LogicalGenerate bottom = new LogicalGenerate<>( + ImmutableList.of(bottomGen), ImmutableList.of(bottomOut), ImmutableList.of(ImmutableList.of("xx")), + ImmutableList.of(BooleanLiteral.TRUE), scan); + + // top has a non-empty conjuncts list -> should NOT merge + SlotReference topOut = new SlotReference("g2", IntegerType.INSTANCE); + Unnest topGen = new Unnest(new IntegerLiteral(2)); + LogicalGenerate top = new LogicalGenerate<>( + ImmutableList.of(topGen), ImmutableList.of(topOut), bottom); + + PlanChecker.from(new ConnectContext(), top).applyBottomUp(new MergeGenerates()) + .printlnAllTree() + .matches(logicalGenerate(logicalGenerate())); + } + + @Test + void testDoNotMergeWhenTopUsesBottomOutput() { + // bottom generate defines an output slot + SlotReference bottomOut = new SlotReference("g_shared", IntegerType.INSTANCE); + Unnest bottomGen = new Unnest(new IntegerLiteral(1)); + LogicalGenerate bottom = new LogicalGenerate<>( + ImmutableList.of(bottomGen), ImmutableList.of(bottomOut), scan); + + // top generator references bottom's output as its argument -> should NOT merge + SlotReference topOut = new SlotReference("g2", IntegerType.INSTANCE); + Unnest topGen = new Unnest(bottomOut); + LogicalGenerate top = new LogicalGenerate<>( + ImmutableList.of(topGen), ImmutableList.of(topOut), bottom); + + PlanChecker.from(new ConnectContext(), top).applyBottomUp(new MergeGenerates()) + .matches(logicalGenerate(logicalGenerate())); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ExpressionUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ExpressionUtilsTest.java index 1e9522558918d9..65e6399e67f4d4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ExpressionUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/ExpressionUtilsTest.java @@ -26,8 +26,22 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.generator.Explode; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeBitmap; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeBitmapOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeMap; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeMapOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.ExplodeOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplode; +import org.apache.doris.nereids.trees.expressions.functions.generator.PosExplodeOuter; +import org.apache.doris.nereids.trees.expressions.functions.generator.Unnest; import org.apache.doris.nereids.trees.expressions.functions.scalar.NonNullable; +import org.apache.doris.nereids.trees.expressions.functions.scalar.ToBitmap; +import org.apache.doris.nereids.trees.expressions.literal.ArrayLiteral; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.MapLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalOdbcScan; @@ -36,6 +50,7 @@ import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -253,6 +268,34 @@ public void testReplaceNullAware() { Assertions.assertEquals(a, replacedExpression2); } + @Test + public void testUnnest() { + List arrayArg = Lists.newArrayList( + new ArrayLiteral(Lists.newArrayList(new IntegerLiteral(1)))); + Unnest unnest = new Unnest(arrayArg, false, false); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof Explode); + unnest = new Unnest(arrayArg, true, false); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof PosExplode); + unnest = new Unnest(arrayArg, false, true); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof ExplodeOuter); + unnest = new Unnest(arrayArg, true, true); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof PosExplodeOuter); + + Map map = Maps.newLinkedHashMap(); + map.put(new IntegerLiteral(0), new BigIntLiteral(0)); + List mapArg = Lists.newArrayList(new MapLiteral(map)); + unnest = new Unnest(mapArg, false, false); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof ExplodeMap); + unnest = new Unnest(mapArg, false, true); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof ExplodeMapOuter); + + List bitmapArg = Lists.newArrayList(new ToBitmap(new IntegerLiteral(1))); + unnest = new Unnest(bitmapArg, false, false); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof ExplodeBitmap); + unnest = new Unnest(bitmapArg, false, true); + Assertions.assertTrue(ExpressionUtils.convertUnnest(unnest) instanceof ExplodeBitmapOuter); + } + private void assertExpect(List originalExpressions, List shuttledExpressions, String... expectExpressions) { From 0e64e32aa6aad86520ac733bf29d89db65025146 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Fri, 26 Dec 2025 11:00:01 +0800 Subject: [PATCH 13/14] add special debug log for debugging unstable p0 case --- be/src/runtime/fragment_mgr.cpp | 16 ++++++++++++++++ regression-test/pipeline/p0/conf/be.conf | 2 +- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp index fed6c3931fed9b..1bc28bdd3dc25b 100644 --- a/be/src/runtime/fragment_mgr.cpp +++ b/be/src/runtime/fragment_mgr.cpp @@ -847,6 +847,22 @@ std::string FragmentMgr::dump_pipeline_tasks(TUniqueId& query_id) { Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params, QuerySource query_source, const FinishCallback& cb, const TPipelineFragmentParamsList& parent) { + auto dbg_str = apache::thrift::ThriftDebugString(params); + if (dbg_str.find("category_ids") != std::string::npos) { + constexpr size_t max_log_size = 30000 - 100; + size_t pos = 0; + size_t total_size = dbg_str.size(); + size_t tmp_size = std::min(max_log_size, total_size); + LOG(WARNING) << "xxxx Query: " << print_id(params.query_id) + << " exec_plan_fragment params size: " << total_size; + while (pos < total_size) { + tmp_size = std::min(max_log_size, total_size - pos); + LOG(WARNING) << "xxxx =========" << std::string(dbg_str.data() + pos, tmp_size); + pos += tmp_size; + } + LOG(WARNING) << "Query: " << print_id(params.query_id) << "query options is " + << apache::thrift::ThriftDebugString(params.query_options).c_str(); + } VLOG_ROW << "Query: " << print_id(params.query_id) << " exec_plan_fragment params is " << apache::thrift::ThriftDebugString(params).c_str(); // sometimes TPipelineFragmentParams debug string is too long and glog diff --git a/regression-test/pipeline/p0/conf/be.conf b/regression-test/pipeline/p0/conf/be.conf index fd11c21f55fd0a..01df493f002f37 100644 --- a/regression-test/pipeline/p0/conf/be.conf +++ b/regression-test/pipeline/p0/conf/be.conf @@ -35,7 +35,7 @@ JEMALLOC_PROF_PRFIX="jemalloc_heap_profile_" # INFO, WARNING, ERROR, FATAL sys_log_level = INFO -sys_log_verbose_modules=fragment_mgr +# sys_log_verbose_modules=dictionary_factory be_port = 9161 webserver_port = 8141 heartbeat_service_port = 9151 From b0538d283d7752ca76828ab9b1a7c9906ba1bc50 Mon Sep 17 00:00:00 2001 From: jacktengg Date: Tue, 30 Dec 2025 12:00:37 +0800 Subject: [PATCH 14/14] test --- .../pipeline/exec/analytic_sink_operator.cpp | 47 ++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 4cd81d4c37effa..51345417465962 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -19,6 +19,7 @@ #include "analytic_sink_operator.h" #include +#include #include #include @@ -280,13 +281,29 @@ bool AnalyticSinkLocalState::_get_next_for_partition(int64_t current_block_rows, bool AnalyticSinkLocalState::_get_next_for_unbounded_range(int64_t current_block_rows, int64_t current_block_base_pos) { + auto& p = _parent->cast(); _update_order_by_range(); + if (p._partition_exprs.size() == 1) { + auto dbg_str = apache::thrift::ThriftDebugString(p._partition_exprs[0]); + if (dbg_str.find("category_ids") != std::string::npos) { + LOG(WARNING) << "xxxx _execute_impl _order_by_pose, is_ended: " + << _order_by_pose.is_ended << ", start: " << _order_by_pose.start + << ", end: " << _order_by_pose.end; + } + } if (!_order_by_pose.is_ended) { DCHECK(!_partition_by_pose.is_ended); _need_more_data = true; return false; } while (_current_row_position < _order_by_pose.end) { + if (p._partition_exprs.size() == 1) { + auto dbg_str = apache::thrift::ThriftDebugString(p._partition_exprs[0]); + if (dbg_str.find("category_ids") != std::string::npos) { + LOG(WARNING) << "xxxx _execute_impl _current_row_position : " + << _current_row_position; + } + } if (_current_row_position == _order_by_pose.start) { _execute_for_function(_partition_by_pose.start, _partition_by_pose.end, _order_by_pose.start, _order_by_pose.end); @@ -342,9 +359,19 @@ bool AnalyticSinkLocalState::_get_next_for_range_between(int64_t current_block_r } Status AnalyticSinkLocalState::_execute_impl() { + auto& p = _parent->cast(); while (_output_block_index < _input_blocks.size()) { { _get_partition_by_end(); + if (p._partition_exprs.size() == 1) { + auto dbg_str = apache::thrift::ThriftDebugString(p._partition_exprs[0]); + if (dbg_str.find("category_ids") != std::string::npos) { + LOG(WARNING) << "xxxx _execute_impl _partition_by_pose, is_ended: " + << _partition_by_pose.is_ended + << ", start: " << _partition_by_pose.start + << ", end: " << _partition_by_pose.end; + } + } // streaming_mode means no need get all parition data, could calculate data when it's arrived if (!_partition_by_pose.is_ended && (!_streaming_mode || _need_more_data)) { _need_more_data = false; @@ -649,7 +676,19 @@ AnalyticSinkOperatorX::AnalyticSinkOperatorX(ObjectPool* pool, int operator_id, _has_window(tnode.analytic_node.__isset.window), _has_range_window(tnode.analytic_node.window.type == TAnalyticWindowType::RANGE), _has_window_start(tnode.analytic_node.window.__isset.window_start), - _has_window_end(tnode.analytic_node.window.__isset.window_end) {} + _has_window_end(tnode.analytic_node.window.__isset.window_end) { + if (_partition_exprs.size() == 1) { + auto dbg_str = apache::thrift::ThriftDebugString(_partition_exprs[0]); + if (dbg_str.find("category_ids") != std::string::npos) { + LOG(WARNING) << "xxxx AnalyticSinkOperatorX:: _is_colocate: " << _is_colocate + << ", _require_bucket_distribution: " << _require_bucket_distribution + << ", _has_window: " << _has_window + << ", _has_range_window: " << _has_range_window + << ", _has_window_start: " << _has_window_start + << ", _has_window_end: " << _has_window_end; + } + } +} Status AnalyticSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(DataSinkOperatorX::init(tnode, state)); @@ -779,6 +818,12 @@ Status AnalyticSinkOperatorX::_add_input_block(doris::RuntimeState* state, if (input_block->rows() <= 0) { return Status::OK(); } + if (_partition_exprs.size() == 1) { + auto dbg_str = apache::thrift::ThriftDebugString(_partition_exprs[0]); + if (dbg_str.find("category_ids") != std::string::npos) { + LOG(WARNING) << "xxxx AnalyticSinkOperatorX::sink block: " << input_block->dump_data(); + } + } auto& local_state = get_local_state(state); local_state._input_block_first_row_positions.emplace_back(local_state._input_total_rows); size_t block_rows = input_block->rows();