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 fdde0446961935..5209d754870034 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 @@ -187,6 +187,10 @@ limitClause | (LIMIT offset=INTEGER_VALUE COMMA limit=INTEGER_VALUE) ; +partitionClause + : PARTITION BY expression (COMMA expression)* + ; + joinType : INNER? | CROSS @@ -316,7 +320,8 @@ primaryExpression | ASTERISK #star | qualifiedName DOT ASTERISK #star | functionIdentifier LEFT_PAREN ((DISTINCT|ALL)? arguments+=expression - (COMMA arguments+=expression)* (ORDER BY sortItem (COMMA sortItem)*)?)? RIGHT_PAREN #functionCall + (COMMA arguments+=expression)* (ORDER BY sortItem (COMMA sortItem)*)?)? RIGHT_PAREN + (OVER windowSpec)? #functionCall | LEFT_PAREN query RIGHT_PAREN #subqueryExpression | ATSIGN identifier #userVariable | DOUBLEATSIGN (kind=(GLOBAL | SESSION) DOT)? identifier #systemVariable @@ -332,6 +337,33 @@ functionIdentifier | LEFT | RIGHT ; +windowSpec + // todo: name for windowRef; we haven't support it + // : name=identifier + // | LEFT_PAREN name=identifier RIGHT_PAREN + : LEFT_PAREN + partitionClause? + sortClause? + windowFrame? + RIGHT_PAREN + ; + +windowFrame + : frameUnits start=frameBoundary + | frameUnits BETWEEN start=frameBoundary AND end=frameBoundary + ; + +frameUnits + : ROWS + | RANGE + ; + +frameBoundary + : UNBOUNDED boundType=(PRECEDING | FOLLOWING) + | boundType=CURRENT ROW + | expression boundType=(PRECEDING | FOLLOWING) + ; + qualifiedName : identifier (DOT identifier)* ; @@ -569,7 +601,6 @@ nonReserved | OUT | OUTER | OUTPUTFORMAT - | OVER | OVERLAPS | OVERLAY | OVERWRITE diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java index f5f1aa53970971..0796f5deed5400 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java @@ -64,7 +64,6 @@ */ public class AnalyticExpr extends Expr { private static final Logger LOG = LoggerFactory.getLogger(AnalyticExpr.class); - private static String NTILE = "NTILE"; private FunctionCallExpr fnCall; private final List partitionExprs; @@ -81,6 +80,7 @@ public class AnalyticExpr extends Expr { // SQL string of this AnalyticExpr before standardization. Returned in toSqlImpl(). private String sqlString; + private static String NTILE = "NTILE"; private static String LEAD = "LEAD"; private static String LAG = "LAG"; private static String FIRSTVALUE = "FIRST_VALUE"; @@ -784,7 +784,7 @@ private void standardize(Analyzer analyzer) throws AnalysisException { } // Reverse the ordering and window for windows ending with UNBOUNDED FOLLOWING, - // and and not starting with UNBOUNDED PRECEDING. + // and not starting with UNBOUNDED PRECEDING. if (window != null && window.getRightBoundary().getType() == BoundaryType.UNBOUNDED_FOLLOWING && window.getLeftBoundary().getType() != BoundaryType.UNBOUNDED_PRECEDING) { @@ -1002,4 +1002,8 @@ private String exprListToDigest(List exprs) { } return Joiner.on(", ").join(strings); } + + @Override + public void finalizeImplForNereids() throws AnalysisException { + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticWindow.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticWindow.java index 4266d0288881eb..47e756578d8ea1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticWindow.java @@ -41,7 +41,7 @@ public class AnalyticWindow { new Boundary(BoundaryType.UNBOUNDED_PRECEDING, null), new Boundary(BoundaryType.CURRENT_ROW, null)); - enum Type { + public enum Type { ROWS("ROWS"), RANGE("RANGE"); @@ -61,7 +61,7 @@ public TAnalyticWindowType toThrift() { } } - enum BoundaryType { + public enum BoundaryType { UNBOUNDED_PRECEDING("UNBOUNDED PRECEDING"), UNBOUNDED_FOLLOWING("UNBOUNDED FOLLOWING"), CURRENT_ROW("CURRENT ROW"), @@ -152,7 +152,7 @@ public Boundary(BoundaryType type, Expr e) { } // c'tor used by clone() - private Boundary(BoundaryType type, Expr e, BigDecimal offsetValue) { + public Boundary(BoundaryType type, Expr e, BigDecimal offsetValue) { Preconditions.checkState( (type.isOffset() && e != null) || (!type.isOffset() && e == null)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java index 2e2bacffc9da68..7ea8ae42873953 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java @@ -297,8 +297,8 @@ public InferPredicateState(InferPredicateState that) { // state shared between all objects of an Analyzer tree // TODO: Many maps here contain properties about tuples, e.g., whether - // a tuple is outer/semi joined, etc. Remove the maps in favor of making - // them properties of the tuple descriptor itself. + // a tuple is outer/semi joined, etc. Remove the maps in favor of making + // them properties of the tuple descriptor itself. private static class GlobalState { private final DescriptorTable descTbl = new DescriptorTable(); private final Env env; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java index 07996451d3794d..4f6d417ac1223c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/AggregateFunction.java @@ -495,6 +495,10 @@ public boolean isAnalyticFn() { return isAnalyticFn; } + public void setIsAnalyticFn(boolean isAnalyticFn) { + this.isAnalyticFn = isAnalyticFn; + } + public boolean isAggregateFn() { return isAggregateFn; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinWindowFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinWindowFunctions.java new file mode 100644 index 00000000000000..07cd376d705e1e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinWindowFunctions.java @@ -0,0 +1,54 @@ +// 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.catalog; + +import org.apache.doris.nereids.trees.expressions.functions.window.DenseRank; +import org.apache.doris.nereids.trees.expressions.functions.window.FirstValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lag; +import org.apache.doris.nereids.trees.expressions.functions.window.LastValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lead; +import org.apache.doris.nereids.trees.expressions.functions.window.Ntile; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber; + +import com.google.common.collect.ImmutableList; + +/** + * Builtin aggregate functions. + * + * Note: Please ensure that this class only has some lists and no procedural code. + * It helps to be clear and concise. + */ +public class BuiltinWindowFunctions implements FunctionHelper { + + public final ImmutableList windowFunctions = ImmutableList.of( + window(DenseRank.class, "dense_rank"), + window(FirstValue.class, "first_value"), + window(Lag.class, "lag"), + window(LastValue.class, "last_value"), + window(Lead.class, "lead"), + window(Ntile.class, "ntile"), + window(Rank.class, "rank"), + window(RowNumber.class, "row_number") + ); + + public static final BuiltinWindowFunctions INSTANCE = new BuiltinWindowFunctions(); + + // Note: Do not add any code here! + private BuiltinWindowFunctions() {} +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionHelper.java index 58c4c91c13ac49..1cb767468b945a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionHelper.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunction; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; +import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; import com.google.common.collect.ImmutableList; @@ -69,6 +70,10 @@ default AggregateFunc agg(Class functionClass) { return new AggregateFunc(functionClass, functionName); } + default WindowFunc window(Class functionClass, String... functionNames) { + return new WindowFunc(functionClass, functionNames); + } + /** * Resolve AggregateFunction class, convert to FunctionBuilder and wrap to AggregateFunc * @param functionClass the AggregateFunction class @@ -129,4 +134,10 @@ public TableGeneratingFunc(Class functionClas super(functionClass, names); } } + + class WindowFunc extends NamedFunc { + public WindowFunc(Class functionClass, String... names) { + super(functionClass, names); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionRegistry.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionRegistry.java index 78dd1ccd01d8bb..68cb4ba6684513 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionRegistry.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionRegistry.java @@ -88,6 +88,7 @@ private void registerBuiltinFunctions(Map> name2Bu FunctionHelper.addFunctions(name2Builders, BuiltinAggregateFunctions.INSTANCE.aggregateFunctions); FunctionHelper.addFunctions(name2Builders, BuiltinTableValuedFunctions.INSTANCE.tableValuedFunctions); FunctionHelper.addFunctions(name2Builders, BuiltinTableGeneratingFunctions.INSTANCE.tableGeneratingFunctions); + FunctionHelper.addFunctions(name2Builders, BuiltinWindowFunctions.INSTANCE.windowFunctions); } public String getCandidateHint(String name, List candidateBuilders) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index e80a0018f5f5ec..b535b08eb5164a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -96,7 +96,6 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions if (explainLevel.isPlanLevel) { return; } - PhysicalPlan physicalPlan = (PhysicalPlan) resultPlan; PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator(); PlanTranslatorContext planTranslatorContext = new PlanTranslatorContext(cascadesContext); @@ -161,7 +160,6 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, Explain return analyzedPlan; } } - // rule-based optimize rewrite(); if (explainLevel == ExplainLevel.REWRITTEN_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { @@ -170,7 +168,6 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, Explain return rewrittenPlan; } } - deriveStats(); if (statementContext.getConnectContext().getSessionVariable().isEnableDPHypOptimizer()) { @@ -182,7 +179,6 @@ public Plan plan(LogicalPlan plan, PhysicalProperties requireProperties, Explain PhysicalPlan physicalPlan = chooseBestPlan(getRoot(), requireProperties); - // post-process physical plan out of memo, just for future use. physicalPlan = postProcess(physicalPlan); if (explainLevel == ExplainLevel.OPTIMIZED_PLAN || explainLevel == ExplainLevel.ALL_PLAN) { optimizedPlan = physicalPlan; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index 776d02a3a823ad..349d9d32173eb4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -76,6 +76,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject; import org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunction; +import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; import org.apache.doris.nereids.trees.expressions.literal.DateTimeLiteral; import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -282,13 +283,58 @@ public Expr visitCast(Cast cast, PlanTranslatorContext context) { @Override public Expr visitInPredicate(InPredicate inPredicate, PlanTranslatorContext context) { List inList = inPredicate.getOptions().stream() - .map(e -> translate(e, context)) + .map(e -> e.accept(this, context)) .collect(Collectors.toList()); return new org.apache.doris.analysis.InPredicate(inPredicate.getCompareExpr().accept(this, context), inList, false); } + @Override + public Expr visitWindowFunction(WindowFunction function, PlanTranslatorContext context) { + // translate argument types from DataType to Type + List catalogArguments = function.getArguments() + .stream() + .map(arg -> arg.accept(this, context)) + .collect(ImmutableList.toImmutableList()); + ImmutableList argTypes = catalogArguments.stream() + .map(arg -> arg.getType()) + .collect(ImmutableList.toImmutableList()); + + // translate argument from List to FunctionParams + List arguments = function.getArguments() + .stream() + .map(arg -> new SlotRef(arg.getDataType().toCatalogDataType(), arg.nullable())) + .collect(ImmutableList.toImmutableList()); + FunctionParams windowFnParams = new FunctionParams(false, arguments); + + // translate isNullable() + NullableMode nullableMode = function.nullable() + ? NullableMode.ALWAYS_NULLABLE + : NullableMode.ALWAYS_NOT_NULLABLE; + + // translate function from WindowFunction to old AggregateFunction + boolean isAnalyticFunction = true; + org.apache.doris.catalog.AggregateFunction catalogFunction = new org.apache.doris.catalog.AggregateFunction( + new FunctionName(function.getName()), argTypes, + function.getDataType().toCatalogDataType(), + function.getDataType().toCatalogDataType(), + function.hasVarArguments(), + null, "", "", null, "", + null, "", null, false, + isAnalyticFunction, false, TFunctionBinaryType.BUILTIN, + true, true, nullableMode + ); + + // generate FunctionCallExpr + boolean isMergeFn = false; + FunctionCallExpr functionCallExpr = + new FunctionCallExpr(catalogFunction, windowFnParams, windowFnParams, isMergeFn, catalogArguments); + functionCallExpr.setIsAnalyticFnCall(true); + return functionCallExpr; + + } + @Override public Expr visitScalarFunction(ScalarFunction function, PlanTranslatorContext context) { List nereidsArguments = adaptFunctionArgumentsForBackends(function); 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 a55b9ef1d2721d..15d4957758ae4f 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 @@ -18,12 +18,17 @@ package org.apache.doris.nereids.glue.translator; import org.apache.doris.analysis.AggregateInfo; +import org.apache.doris.analysis.AnalyticWindow; import org.apache.doris.analysis.BaseTableRef; import org.apache.doris.analysis.BinaryPredicate; +import org.apache.doris.analysis.BoolLiteral; +import org.apache.doris.analysis.CompoundPredicate; import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.FunctionCallExpr; import org.apache.doris.analysis.GroupByClause.GroupingType; import org.apache.doris.analysis.GroupingInfo; +import org.apache.doris.analysis.IsNullPredicate; +import org.apache.doris.analysis.OrderByElement; import org.apache.doris.analysis.SlotDescriptor; import org.apache.doris.analysis.SlotId; import org.apache.doris.analysis.SlotRef; @@ -48,6 +53,7 @@ import org.apache.doris.nereids.properties.DistributionSpecReplicated; import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; @@ -55,9 +61,13 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.VirtualSlotReference; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.plans.AggMode; import org.apache.doris.nereids.trees.plans.AggPhase; @@ -77,6 +87,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalIntersect; import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; +import org.apache.doris.nereids.trees.plans.physical.PhysicalLocalQuickSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; @@ -90,6 +101,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.util.ExpressionUtils; @@ -97,6 +109,7 @@ import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; import org.apache.doris.planner.AggregationNode; +import org.apache.doris.planner.AnalyticEvalNode; import org.apache.doris.planner.AssertNumRowsNode; import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.EmptySetNode; @@ -199,7 +212,6 @@ public PlanFragment translatePlan(PhysicalPlan physicalPlan, PlanTranslatorConte return rootFragment; } - /** * Translate Agg. */ @@ -610,6 +622,99 @@ public PlanFragment visitPhysicalQuickSort(PhysicalQuickSort sor return mergeFragment; } + @Override + public PlanFragment visitPhysicalWindow(PhysicalWindow physicalWindow, + PlanTranslatorContext context) { + PlanFragment inputPlanFragment = physicalWindow.child(0).accept(this, context); + + // 1. translate to old optimizer variable + // variable in Nereids + WindowFrameGroup windowFrameGroup = physicalWindow.getWindowFrameGroup(); + List partitionKeyList = Lists.newArrayList(windowFrameGroup.getPartitionKeys()); + List orderKeyList = windowFrameGroup.getOrderKeys(); + List windowFunctionList = windowFrameGroup.getGroups(); + WindowFrame windowFrame = windowFrameGroup.getWindowFrame(); + + // partition by clause + List partitionExprs = partitionKeyList.stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toList()); + + // order by clause + List orderByElements = orderKeyList.stream() + .map(orderKey -> new OrderByElement( + ExpressionTranslator.translate(orderKey.child(), context), + orderKey.isAsc(), orderKey.isNullFirst())) + .collect(Collectors.toList()); + + // function calls + List analyticFnCalls = windowFunctionList.stream() + .map(e -> { + Expression function = e.child(0).child(0); + if (function instanceof AggregateFunction) { + AggregateParam param = AggregateParam.localResult(); + function = new AggregateExpression((AggregateFunction) function, param); + } + return ExpressionTranslator.translate(function, context); + }) + .map(FunctionCallExpr.class::cast) + .map(fnCall -> { + fnCall.setIsAnalyticFnCall(true); + ((org.apache.doris.catalog.AggregateFunction) fnCall.getFn()).setIsAnalyticFn(true); + return fnCall; + }) + .collect(Collectors.toList()); + + // analytic window + AnalyticWindow analyticWindow = physicalWindow.translateWindowFrame(windowFrame, context); + + // 2. get bufferedTupleDesc from SortNode and compute isNullableMatched + Map bufferedSlotRefForWindow = getBufferedSlotRefForWindow(windowFrameGroup, context); + TupleDescriptor bufferedTupleDesc = context.getBufferedTupleForWindow(); + + // generate predicates to check if the exprs of partitionKeys and orderKeys have matched isNullable between + // sortNode and analyticNode + Expr partitionExprsIsNullableMatched = partitionExprs.isEmpty() ? null : windowExprsHaveMatchedNullable( + partitionKeyList, partitionExprs, bufferedSlotRefForWindow); + + Expr orderElementsIsNullableMatched = orderByElements.isEmpty() ? null : windowExprsHaveMatchedNullable( + orderKeyList.stream().map(order -> order.child()).collect(Collectors.toList()), + orderByElements.stream().map(order -> order.getExpr()).collect(Collectors.toList()), + bufferedSlotRefForWindow); + + // 3. generate tupleDesc + List windowSlotList = windowFunctionList.stream() + .map(NamedExpression::toSlot) + .collect(Collectors.toList()); + TupleDescriptor outputTupleDesc = generateTupleDesc(windowSlotList, null, context); + TupleDescriptor intermediateTupleDesc = outputTupleDesc; + + // 4. generate AnalyticEvalNode + AnalyticEvalNode analyticEvalNode = new AnalyticEvalNode( + context.nextPlanNodeId(), + inputPlanFragment.getPlanRoot(), + analyticFnCalls, + partitionExprs, + orderByElements, + analyticWindow, + intermediateTupleDesc, + outputTupleDesc, + partitionExprsIsNullableMatched, + orderElementsIsNullableMatched, + bufferedTupleDesc + ); + + if (partitionExprs.isEmpty() && orderByElements.isEmpty()) { + if (inputPlanFragment.isPartitioned()) { + inputPlanFragment = createParentFragment(inputPlanFragment, DataPartition.UNPARTITIONED, context); + } + } else { + analyticEvalNode.setNumInstances(inputPlanFragment.getPlanRoot().getNumInstances()); + } + inputPlanFragment.addPlanRoot(analyticEvalNode); + return inputPlanFragment; + } + @Override public PlanFragment visitPhysicalTopN(PhysicalTopN topN, PlanTranslatorContext context) { PlanFragment childFragment = visitAbstractPhysicalSort(topN, context); @@ -628,6 +733,66 @@ public PlanFragment visitPhysicalTopN(PhysicalTopN topN, PlanTra return mergeFragment; } + /** + * Currently this function is added for PhysicalWindow's requiredProperties only. + * The code is equal with visitAbstractPhysicalSort(), except step 2:check if it is a sort for PhysicalWindow + * and setBufferedTupleForWindow() in step 3. + */ + @Override + public PlanFragment visitPhysicalLocalQuickSort(PhysicalLocalQuickSort sort, + PlanTranslatorContext context) { + PlanFragment childFragment = sort.child(0).accept(this, context); + List oldOrderingExprList = Lists.newArrayList(); + List ascOrderList = Lists.newArrayList(); + List nullsFirstParamList = Lists.newArrayList(); + List orderKeyList = sort.getOrderKeys(); + // 1. Get previous slotRef + orderKeyList.forEach(k -> { + oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrderList.add(k.isAsc()); + nullsFirstParamList.add(k.isNullFirst()); + }); + List sortTupleOutputList = new ArrayList<>(); + List outputList = sort.getOutput(); + outputList.forEach(k -> { + sortTupleOutputList.add(ExpressionTranslator.translate(k, context)); + }); + + // 2. check if it is a sort for PhysicalWindow + PlanFragment currentFragment; + boolean useTopN = true; + if (childFragment.getPlanRoot() instanceof ExchangeNode && sort.child() instanceof PhysicalDistribute) { + PhysicalDistribute physicalDistribute = (PhysicalDistribute) sort.child(); + DataPartition dataPartition = hashSpecToDataPartition(physicalDistribute, context); + + ExchangeNode exchangeNode = (ExchangeNode) childFragment.getPlanRoot(); + currentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition); + childFragment.setOutputPartition(dataPartition); + childFragment.setPlanRoot(exchangeNode.getChild(0)); + childFragment.setDestination(exchangeNode); + context.addPlanFragment(currentFragment); + useTopN = false; + } else { + currentFragment = childFragment; + } + + // 3. Generate new Tuple and get current slotRef for newOrderingExprList + List newOrderingExprList = Lists.newArrayList(); + TupleDescriptor tupleDesc = generateTupleDesc(outputList, orderKeyList, newOrderingExprList, context, null); + context.setBufferedTupleForWindow(tupleDesc); + + // 4. fill in SortInfo members + SortInfo sortInfo = new SortInfo(newOrderingExprList, ascOrderList, nullsFirstParamList, tupleDesc); + PlanNode childNode = currentFragment.getPlanRoot(); + SortNode sortNode = new SortNode(context.nextPlanNodeId(), childNode, sortInfo, useTopN); + sortNode.finalizeForNereids(tupleDesc, sortTupleOutputList, oldOrderingExprList); + if (sort.getStats() != null) { + sortNode.setCardinality((long) sort.getStats().getRowCount()); + } + currentFragment.addPlanRoot(sortNode); + return currentFragment; + } + @Override public PlanFragment visitAbstractPhysicalSort( AbstractPhysicalSort sort, @@ -637,7 +802,7 @@ public PlanFragment visitAbstractPhysicalSort( List ascOrderList = Lists.newArrayList(); List nullsFirstParamList = Lists.newArrayList(); List orderKeyList = sort.getOrderKeys(); - // 1.Get previous slotRef + // 1. Get previous slotRef orderKeyList.forEach(k -> { oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); ascOrderList.add(k.isAsc()); @@ -1744,6 +1909,68 @@ private Optional toDataPartition(PhysicalDistribute distribute, } } + private Map getBufferedSlotRefForWindow(WindowFrameGroup windowFrameGroup, + PlanTranslatorContext context) { + Map bufferedSlotRefForWindow = context.getBufferedSlotRefForWindow(); + + // set if absent + windowFrameGroup.getPartitionKeys().stream() + .map(NamedExpression.class::cast) + .forEach(expression -> { + ExprId exprId = expression.getExprId(); + bufferedSlotRefForWindow.putIfAbsent(exprId, context.findSlotRef(exprId)); + }); + windowFrameGroup.getOrderKeys().stream() + .map(ok -> ok.child()) + .map(NamedExpression.class::cast) + .forEach(expression -> { + ExprId exprId = expression.getExprId(); + bufferedSlotRefForWindow.putIfAbsent(exprId, context.findSlotRef(exprId)); + }); + return bufferedSlotRefForWindow; + } + + private Expr windowExprsHaveMatchedNullable(List expressions, List exprs, + Map bufferedSlotRef) { + Map exprIdToExpr = Maps.newHashMap(); + for (int i = 0; i < expressions.size(); i++) { + NamedExpression expression = (NamedExpression) expressions.get(i); + exprIdToExpr.put(expression.getExprId(), exprs.get(i)); + } + return windowExprsHaveMatchedNullable(exprIdToExpr, bufferedSlotRef, expressions, 0, expressions.size()); + } + + private Expr windowExprsHaveMatchedNullable(Map exprIdToExpr, Map exprIdToSlotRef, + List expressions, int i, int size) { + if (i > size - 1) { + return new BoolLiteral(true); + } + + ExprId exprId = ((NamedExpression) expressions.get(i)).getExprId(); + Expr lhs = exprIdToExpr.get(exprId); + Expr rhs = exprIdToSlotRef.get(exprId); + + Expr bothNull = new CompoundPredicate(CompoundPredicate.Operator.AND, + new IsNullPredicate(lhs, false), new IsNullPredicate(rhs, false)); + Expr lhsEqRhsNotNull = new CompoundPredicate(CompoundPredicate.Operator.AND, + new CompoundPredicate(CompoundPredicate.Operator.AND, + new IsNullPredicate(lhs, true), new IsNullPredicate(rhs, true)), + new BinaryPredicate(BinaryPredicate.Operator.EQ, lhs, rhs)); + + Expr remainder = windowExprsHaveMatchedNullable(exprIdToExpr, exprIdToSlotRef, expressions, i + 1, size); + return new CompoundPredicate(CompoundPredicate.Operator.AND, + new CompoundPredicate(CompoundPredicate.Operator.OR, bothNull, lhsEqRhsNotNull), remainder); + } + + private DataPartition hashSpecToDataPartition(PhysicalDistribute distribute, PlanTranslatorContext context) { + Preconditions.checkState(distribute.getDistributionSpec() instanceof DistributionSpecHash); + DistributionSpecHash hashSpec = (DistributionSpecHash) distribute.getDistributionSpec(); + List partitions = hashSpec.getOrderedShuffledColumns().stream() + .map(exprId -> context.findSlotRef(exprId)) + .collect(Collectors.toList()); + return new DataPartition(TPartitionType.HASH_PARTITIONED, partitions); + } + private static class SetOperationResult { private final List> resultExpressions; private final List> constExpressions; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java index ab5a181646590e..da85dceb7961ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PlanTranslatorContext.java @@ -78,6 +78,9 @@ public class PlanTranslatorContext { private final IdentityHashMap firstAggInFragment = new IdentityHashMap<>(); + private final Map bufferedSlotRefForWindow = Maps.newHashMap(); + private TupleDescriptor bufferedTupleForWindow = null; + public PlanTranslatorContext(CascadesContext ctx) { this.translator = new RuntimeFilterTranslator(ctx.getRuntimeFilterContext()); } @@ -148,6 +151,18 @@ public void setFirstAggregateInFragment(PlanFragment planFragment, PhysicalHashA firstAggInFragment.put(planFragment, aggregate); } + public Map getBufferedSlotRefForWindow() { + return bufferedSlotRefForWindow; + } + + public TupleDescriptor getBufferedTupleForWindow() { + return bufferedTupleForWindow; + } + + public void setBufferedTupleForWindow(TupleDescriptor bufferedTupleForWindow) { + this.bufferedTupleForWindow = bufferedTupleForWindow; + } + /** * Create SlotDesc and add it to the mappings from expression to the stales expr. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/NereidsRewriteJobExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/NereidsRewriteJobExecutor.java index 336096b74c4f40..1b9d74e2445d81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/NereidsRewriteJobExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/batch/NereidsRewriteJobExecutor.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.rules.mv.SelectMaterializedIndexWithoutAggregate; import org.apache.doris.nereids.rules.rewrite.logical.AdjustNullable; import org.apache.doris.nereids.rules.rewrite.logical.BuildAggForUnion; +import org.apache.doris.nereids.rules.rewrite.logical.CheckAndStandardizeWindowFunctionAndFrame; import org.apache.doris.nereids.rules.rewrite.logical.ColumnPruning; import org.apache.doris.nereids.rules.rewrite.logical.CountDistinctRewrite; import org.apache.doris.nereids.rules.rewrite.logical.EliminateAggregate; @@ -39,6 +40,7 @@ import org.apache.doris.nereids.rules.rewrite.logical.EliminateLimit; import org.apache.doris.nereids.rules.rewrite.logical.EliminateOrderByConstant; import org.apache.doris.nereids.rules.rewrite.logical.EliminateUnnecessaryProject; +import org.apache.doris.nereids.rules.rewrite.logical.ExtractAndNormalizeWindowExpression; import org.apache.doris.nereids.rules.rewrite.logical.ExtractFilterFromCrossJoin; import org.apache.doris.nereids.rules.rewrite.logical.ExtractSingleTableExpressionFromDisjunction; import org.apache.doris.nereids.rules.rewrite.logical.FindHashConditionForJoin; @@ -90,6 +92,11 @@ public NereidsRewriteJobExecutor(CascadesContext cascadesContext) { .add(bottomUpBatch(ImmutableList.of(new AdjustAggregateNullableForEmptySet()))) .add(topDownBatch(ImmutableList.of(new EliminateGroupByConstant()))) .add(topDownBatch(ImmutableList.of(new NormalizeAggregate()))) + .add(topDownBatch(ImmutableList.of(new ExtractAndNormalizeWindowExpression()))) + //execute NormalizeAggregate() again to resolve nested AggregateFunctions in WindowExpression, + //e.g. sum(sum(c1)) over(partition by avg(c1)) + .add(topDownBatch(ImmutableList.of(new NormalizeAggregate()))) + .add(topDownBatch(ImmutableList.of(new CheckAndStandardizeWindowFunctionAndFrame()))) .add(topDownBatch(RuleSet.PUSH_DOWN_FILTERS, false)) .add(visitorJob(RuleType.INFER_PREDICATES, new InferPredicates())) .add(topDownBatch(ImmutableList.of(new ExtractFilterFromCrossJoin()))) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java index df1596f1892794..078f8e8ce70754 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java @@ -264,7 +264,7 @@ private Group init(Plan plan) { * +---------------------------------------+-----------------------------------+--------------------------------+ * | case 3: | | | * | if targetGroup is not null | true | new group expression | - * | and same group expression not exits | | | + * | and same group expression not exist | | | * +---------------------------------------+-----------------------------------+--------------------------------+ * | case 4: | | | * | if targetGroup is not null and not | true | new group expression | @@ -382,7 +382,7 @@ private List rewriteChildrenPlansToGroups(Plan plan, Group targetGroup) { private void validateRewriteChildGroup(Group childGroup, Group targetGroup) { /* * 'A => B(A)' is invalid equivalent transform because of dead loop. - * see 'MemoRewriteTest.a2ba()' + * see 'MemoTest.a2ba()' */ if (childGroup == targetGroup) { throw new IllegalStateException("Can not add plan which is ancestor of the target plan"); 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 1fcc5b743fe4ec..185e74368b3fe0 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 @@ -100,6 +100,8 @@ import org.apache.doris.nereids.DorisParser.UnitIdentifierContext; import org.apache.doris.nereids.DorisParser.UserVariableContext; import org.apache.doris.nereids.DorisParser.WhereClauseContext; +import org.apache.doris.nereids.DorisParser.WindowFrameContext; +import org.apache.doris.nereids.DorisParser.WindowSpecContext; import org.apache.doris.nereids.DorisParserBaseVisitor; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundAlias; @@ -148,6 +150,8 @@ import org.apache.doris.nereids.trees.expressions.TVFProperties; import org.apache.doris.nereids.trees.expressions.TimestampArithmetic; import org.apache.doris.nereids.trees.expressions.WhenClause; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +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.agg.GroupConcat; @@ -247,6 +251,8 @@ import java.util.Locale; import java.util.Map; import java.util.Optional; +import java.util.function.Supplier; +import java.util.stream.Collectors; /** * Build a logical plan tree with unbounded nodes. @@ -342,13 +348,12 @@ private LogicalPlan withCte(LogicalPlan plan, CteContext ctx) { public LogicalSubQueryAlias visitAliasQuery(AliasQueryContext ctx) { return ParserUtils.withOrigin(ctx, () -> { LogicalPlan queryPlan = plan(ctx.query()); - List columnNames = null; - if (ctx.columnAliases() != null) { - columnNames = ctx.columnAliases().identifier().stream() + Optional> columnNames = optionalVisit(ctx.columnAliases(), () -> + ctx.columnAliases().identifier().stream() .map(RuleContext::getText) - .collect(ImmutableList.toImmutableList()); - } - return new LogicalSubQueryAlias<>(ctx.identifier().getText(), Optional.ofNullable(columnNames), queryPlan); + .collect(ImmutableList.toImmutableList()) + ); + return new LogicalSubQueryAlias<>(ctx.identifier().getText(), columnNames, queryPlan); }); } @@ -955,15 +960,105 @@ public Expression visitFunctionCall(DorisParser.FunctionCallContext ctx) { if (!unboundStars.get(0).getQualifier().isEmpty()) { throw new ParseException("'*' can not has qualifier: " + unboundStars.size(), ctx); } + if (ctx.windowSpec() != null) { + // todo: support count(*) as window function + throw new ParseException( + "COUNT(*) isn't supported as window function; can use COUNT(col)", ctx); + } return new Count(); } throw new ParseException("'*' can only be used in conjunction with COUNT: " + functionName, ctx); } else { - return new UnboundFunction(functionName, isDistinct, params); + UnboundFunction function = new UnboundFunction(functionName, isDistinct, params); + if (ctx.windowSpec() != null) { + if (isDistinct) { + throw new ParseException("DISTINCT not allowed in analytic function: " + functionName, ctx); + } + return withWindowSpec(ctx.windowSpec(), function); + } + return function; } }); } + /** + * deal with window function definition + */ + private WindowExpression withWindowSpec(WindowSpecContext ctx, Expression function) { + List partitionKeyList = Lists.newArrayList(); + if (ctx.partitionClause() != null) { + partitionKeyList = visit(ctx.partitionClause().expression(), Expression.class); + } + + List orderKeyList = Lists.newArrayList(); + if (ctx.sortClause() != null) { + orderKeyList = visit(ctx.sortClause().sortItem(), OrderKey.class).stream() + .map(orderKey -> new OrderExpression(orderKey)) + .collect(Collectors.toList()); + } + + if (ctx.windowFrame() != null) { + return new WindowExpression(function, partitionKeyList, orderKeyList, withWindowFrame(ctx.windowFrame())); + } + return new WindowExpression(function, partitionKeyList, orderKeyList); + } + + /** + * deal with optional expressions + */ + private Optional optionalVisit(T ctx, Supplier func) { + return Optional.ofNullable(ctx).map(a -> func.get()); + } + + /** + * deal with window frame + */ + private WindowFrame withWindowFrame(WindowFrameContext ctx) { + WindowFrame.FrameUnitsType frameUnitsType = WindowFrame.FrameUnitsType.valueOf( + ctx.frameUnits().getText().toUpperCase()); + WindowFrame.FrameBoundary leftBoundary = withFrameBound(ctx.start); + if (ctx.end != null) { + WindowFrame.FrameBoundary rightBoundary = withFrameBound(ctx.end); + return new WindowFrame(frameUnitsType, leftBoundary, rightBoundary); + } + return new WindowFrame(frameUnitsType, leftBoundary); + } + + private WindowFrame.FrameBoundary withFrameBound(DorisParser.FrameBoundaryContext ctx) { + Optional expression = Optional.empty(); + if (ctx.expression() != null) { + expression = Optional.of(getExpression(ctx.expression())); + // todo: use isConstant() to resolve Function in expression; currently we only + // support literal expression + if (!expression.get().isLiteral()) { + throw new ParseException("Unsupported expression in WindowFrame : " + expression, ctx); + } + } + + WindowFrame.FrameBoundType frameBoundType = null; + switch (ctx.boundType.getType()) { + case DorisParser.PRECEDING: + if (ctx.UNBOUNDED() != null) { + frameBoundType = WindowFrame.FrameBoundType.UNBOUNDED_PRECEDING; + } else { + frameBoundType = WindowFrame.FrameBoundType.PRECEDING; + } + break; + case DorisParser.CURRENT: + frameBoundType = WindowFrame.FrameBoundType.CURRENT_ROW; + break; + case DorisParser.FOLLOWING: + if (ctx.UNBOUNDED() != null) { + frameBoundType = WindowFrame.FrameBoundType.UNBOUNDED_FOLLOWING; + } else { + frameBoundType = WindowFrame.FrameBoundType.FOLLOWING; + } + break; + default: + } + return new WindowFrame.FrameBoundary(expression, frameBoundType); + } + @Override public Expression visitInterval(IntervalContext ctx) { return new Interval(getExpression(ctx.value), visitUnitIdentifier(ctx.unit)); @@ -1461,7 +1556,7 @@ private LogicalPlan withProjection(LogicalPlan input, SelectColumnClauseContext expressions, input, isDistinct); } else { List projects = getNamedExpressions(selectCtx.namedExpressionSeq()); - return new LogicalProject<>(projects, ImmutableList.of(), input, isDistinct); + return new LogicalProject<>(projects, Collections.emptyList(), input, isDistinct); } } }); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java index d4c9c24c06293c..54f2e6cb9de808 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java @@ -81,6 +81,10 @@ public static PhysicalProperties createHash(DistributionSpecHash distributionSpe return new PhysicalProperties(distributionSpecHash); } + public PhysicalProperties withOrderSpec(OrderSpec orderSpec) { + return new PhysicalProperties(distributionSpec, orderSpec); + } + // Current properties satisfies other properties. public boolean satisfy(PhysicalProperties other) { return orderSpec.satisfy(other.orderSpec) && distributionSpec.satisfy(other.distributionSpec); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java index 5a5101011f9f14..08f0fe1b8d532f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java @@ -90,7 +90,7 @@ public Void visit(Plan plan, PlanContext context) { for (int i = context.getGroupExpression().arity(); i > 0; --i) { requiredPropertyList.add(PhysicalProperties.ANY); } - requestPropertyToChildren.add(requiredPropertyList); + addRequestPropertyToChildren(requiredPropertyList); return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index b87aff3374ad3f..5483c487b9f4d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -46,6 +46,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalTVFRelationToPhysicalTVFRelation; import org.apache.doris.nereids.rules.implementation.LogicalTopNToPhysicalTopN; import org.apache.doris.nereids.rules.implementation.LogicalUnionToPhysicalUnion; +import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; import org.apache.doris.nereids.rules.rewrite.logical.EliminateOuterJoin; import org.apache.doris.nereids.rules.rewrite.logical.MergeFilters; import org.apache.doris.nereids.rules.rewrite.logical.MergeGenerates; @@ -116,6 +117,7 @@ public class RuleSet { .add(new LogicalFileScanToPhysicalFileScan()) .add(new LogicalProjectToPhysicalProject()) .add(new LogicalLimitToPhysicalLimit()) + .add(new LogicalWindowToPhysicalWindow()) .add(new LogicalSortToPhysicalQuickSort()) .add(new LogicalTopNToPhysicalTopN()) .add(new LogicalAssertNumRowsToPhysicalAssertNumRows()) 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 b570a1c4665ff2..85d9123ce59c1d 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 @@ -70,7 +70,6 @@ public enum RuleType { PROJECT_WITH_DISTINCT_TO_AGGREGATE(RuleTypeClass.REWRITE), AVG_DISTINCT_TO_SUM_DIV_COUNT(RuleTypeClass.REWRITE), REGISTER_CTE(RuleTypeClass.REWRITE), - RELATION_AUTHENTICATION(RuleTypeClass.VALIDATION), ADJUST_NULLABLE_FOR_PROJECT_SLOT(RuleTypeClass.REWRITE), @@ -92,6 +91,8 @@ public enum RuleType { // rewrite rules NORMALIZE_AGGREGATE(RuleTypeClass.REWRITE), NORMALIZE_REPEAT(RuleTypeClass.REWRITE), + EXTRACT_AND_NORMALIZE_WINDOW_EXPRESSIONS(RuleTypeClass.REWRITE), + CHECK_AND_STANDARDIZE_WINDOW_FUNCTION_AND_FRAME(RuleTypeClass.REWRITE), AGGREGATE_DISASSEMBLE(RuleTypeClass.REWRITE), DISTINCT_AGGREGATE_DISASSEMBLE(RuleTypeClass.REWRITE), ELIMINATE_UNNECESSARY_PROJECT(RuleTypeClass.REWRITE), @@ -254,6 +255,7 @@ public enum RuleType { LOGICAL_EXCEPT_TO_PHYSICAL_EXCEPT(RuleTypeClass.IMPLEMENTATION), LOGICAL_INTERSECT_TO_PHYSICAL_INTERSECT(RuleTypeClass.IMPLEMENTATION), LOGICAL_GENERATE_TO_PHYSICAL_GENERATE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_WINDOW_TO_PHYSICAL_WINDOW_RULE(RuleTypeClass.IMPLEMENTATION), IMPLEMENTATION_SENTINEL(RuleTypeClass.IMPLEMENTATION), LOGICAL_SEMI_JOIN_SEMI_JOIN_TRANPOSE_PROJECT(RuleTypeClass.EXPLORATION), 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 6d56a875796870..8b0ec34e7c8cc2 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 @@ -509,8 +509,7 @@ public List buildRules() { ); } - private Plan bindSort( - LogicalSort sort, Plan plan, CascadesContext ctx) { + private Plan bindSort(LogicalSort sort, Plan plan, CascadesContext ctx) { // 1. We should deduplicate the slots, otherwise the binding process will fail due to the // ambiguous slots exist. // 2. try to bound order-key with agg output, if failed, try to bound with output of agg.child diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ProjectToGlobalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ProjectToGlobalAggregate.java index 15d341a6a9b1ef..5a798cf7bc37c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ProjectToGlobalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ProjectToGlobalAggregate.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -59,7 +60,8 @@ public Rule build() { } private boolean hasNonWindowedAggregateFunction(Expression expression) { - // TODO: exclude windowed aggregate function - return expression.anyMatch(AggregateFunction.class::isInstance); + return expression.anyMatch(WindowExpression.class::isInstance) + ? false + : expression.anyMatch(AggregateFunction.class::isInstance); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/WindowFunctionChecker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/WindowFunctionChecker.java new file mode 100644 index 00000000000000..5ee0a47f7a5f0d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/WindowFunctionChecker.java @@ -0,0 +1,438 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.analysis; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundType; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundary; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameUnitsType; +import org.apache.doris.nereids.trees.expressions.functions.window.DenseRank; +import org.apache.doris.nereids.trees.expressions.functions.window.FirstOrLastValue; +import org.apache.doris.nereids.trees.expressions.functions.window.FirstValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lag; +import org.apache.doris.nereids.trees.expressions.functions.window.LastValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lead; +import org.apache.doris.nereids.trees.expressions.functions.window.Ntile; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; +import org.apache.doris.nereids.util.TypeCoercionUtils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Check and standardize Window expression: + * + * step 1: checkWindowBeforeFunc(): + * general checking for WindowFrame, including check OrderKeyList, set default right boundary, check offset if exists, + * check correctness of boundaryType + * step 2: checkWindowFunction(): + * check window function, and different function has different checking rules . + * If window frame not exits, set a unique default window frame according to their function type. + * step 3: checkWindowAfterFunc(): + * reverse window if necessary (just for first_value() and last_value()), and add a general default + * window frame (RANGE between UNBOUNDED PRECEDING and CURRENT ROW) + */ +public class WindowFunctionChecker extends DefaultExpressionVisitor { + + private WindowExpression windowExpression; + + public WindowFunctionChecker(WindowExpression window) { + this.windowExpression = window; + } + + public WindowExpression getWindow() { + return windowExpression; + } + + /** + * step 1: check windowFrame in window; + */ + public void checkWindowBeforeFunc() { + windowExpression.getWindowFrame().ifPresent(this::checkWindowFrameBeforeFunc); + } + + /** + * step 2: check windowFunction in window + */ + public Expression checkWindowFunction() { + // todo: visitNtile() + + // in checkWindowFrameBeforeFunc() we have confirmed that both left and right boundary are set as long as + // windowFrame exists, therefore in all following visitXXX functions we don't need to check whether the right + // boundary is null. + return windowExpression.accept(this, null); + } + + /** + * step 3: check window + */ + public void checkWindowAfterFunc() { + Optional windowFrame = windowExpression.getWindowFrame(); + if (windowFrame.isPresent()) { + // reverse windowFrame + checkWindowFrameAfterFunc(windowFrame.get()); + } else { + setDefaultWindowFrameAfterFunc(); + } + } + + /* ******************************************************************************************** + * methods for step 1 + * ******************************************************************************************** */ + + /** + * + * if WindowFrame doesn't have right boundary, we will set it a default one(current row); + * but if WindowFrame itself doesn't exist, we will keep it null still. + * + * Basic exception cases: + * 0. WindowFrame != null, but OrderKeyList == null + * + * WindowFrame EXCEPTION cases: + * 1. (unbounded following, xxx) || (offset following, !following) + * 2. (xxx, unbounded preceding) || (!preceding, offset preceding) + * 3. RANGE && ( (offset preceding, xxx) || (xxx, offset following) || (current row, current row) ) + * + * WindowFrame boundOffset check: + * 4. check value of boundOffset: Literal; Positive; Integer (for ROWS) or Numeric (for RANGE) + * 5. check that boundOffset of left <= boundOffset of right + */ + private void checkWindowFrameBeforeFunc(WindowFrame windowFrame) { + // case 0 + if (windowExpression.getOrderKeys().isEmpty()) { + throw new AnalysisException("WindowFrame clause requires OrderBy clause"); + } + + // set default rightBoundary + if (windowFrame.getRightBoundary().isNull()) { + windowFrame = windowFrame.withRightBoundary(FrameBoundary.newCurrentRowBoundary()); + } + FrameBoundary left = windowFrame.getLeftBoundary(); + FrameBoundary right = windowFrame.getRightBoundary(); + + // case 1 + if (left.getFrameBoundType() == FrameBoundType.UNBOUNDED_FOLLOWING) { + throw new AnalysisException("WindowFrame in any window function cannot use " + + "UNBOUNDED FOLLOWING as left boundary"); + } + if (left.getFrameBoundType() == FrameBoundType.FOLLOWING && !right.asFollowing()) { + throw new AnalysisException("WindowFrame with FOLLOWING left boundary requires " + + "UNBOUNDED FOLLOWING or FOLLOWING right boundary"); + } + + // case 2 + if (right.getFrameBoundType() == FrameBoundType.UNBOUNDED_PRECEDING) { + throw new AnalysisException("WindowFrame in any window function cannot use " + + "UNBOUNDED PRECEDING as right boundary"); + } + if (right.getFrameBoundType() == FrameBoundType.PRECEDING && !left.asPreceding()) { + throw new AnalysisException("WindowFrame with PRECEDING right boundary requires " + + "UNBOUNDED PRECEDING or PRECEDING left boundary"); + } + + // case 3 + // this case will be removed when RANGE with offset boundaries is supported + if (windowFrame.getFrameUnits() == FrameUnitsType.RANGE) { + if (left.hasOffset() || right.hasOffset() + || (left.getFrameBoundType() == FrameBoundType.CURRENT_ROW + && right.getFrameBoundType() == FrameBoundType.CURRENT_ROW)) { + throw new AnalysisException("WindowFrame with RANGE must use both UNBOUNDED boundary or " + + "one UNBOUNDED boundary and one CURRENT ROW"); + } + } + + // case 4 + if (left.hasOffset()) { + checkFrameBoundOffset(left); + } + if (right.hasOffset()) { + checkFrameBoundOffset(right); + } + + // case 5 + // check correctness of left boundary and right boundary + if (left.hasOffset() && right.hasOffset()) { + double leftOffsetValue = ((Literal) left.getBoundOffset().get()).getDouble(); + double rightOffsetValue = ((Literal) right.getBoundOffset().get()).getDouble(); + if (left.asPreceding() && right.asPreceding()) { + Preconditions.checkArgument(leftOffsetValue >= rightOffsetValue, "WindowFrame with " + + "PRECEDING boundary requires that leftBoundOffset >= rightBoundOffset"); + } else if (left.asFollowing() && right.asFollowing()) { + Preconditions.checkArgument(leftOffsetValue <= rightOffsetValue, "WindowFrame with " + + "FOLLOWING boundary requires that leftBoundOffset >= rightBoundOffset"); + } + } + + windowExpression = windowExpression.withWindowFrame(windowFrame); + } + + /** + * check boundOffset of FrameBoundary if it exists: + * 1 boundOffset should be Literal, but this restriction can be removed after completing FoldConstant + * 2 boundOffset should be positive + * 2 boundOffset should be a positive INTEGER if FrameUnitsType == ROWS + * 3 boundOffset should be a positive INTEGER or DECIMAL if FrameUnitsType == RANGE + */ + private void checkFrameBoundOffset(FrameBoundary frameBoundary) { + Expression offset = frameBoundary.getBoundOffset().get(); + + // case 1 + Preconditions.checkArgument(offset.isLiteral(), "BoundOffset of WindowFrame must be Literal"); + + // case 2 + boolean isPositive = ((Literal) offset).getDouble() > 0; + Preconditions.checkArgument(isPositive, "BoundOffset of WindowFrame must be positive"); + + // case 3 + FrameUnitsType frameUnits = windowExpression.getWindowFrame().get().getFrameUnits(); + if (frameUnits == FrameUnitsType.ROWS) { + Preconditions.checkArgument(offset.getDataType().isIntegralType(), + "BoundOffset of ROWS WindowFrame must be an Integer"); + } + + // case 4 + if (frameUnits == FrameUnitsType.RANGE) { + Preconditions.checkArgument(offset.getDataType().isNumericType(), + "BoundOffset of RANGE WindowFrame must be an Integer or Decimal"); + } + } + + /* ******************************************************************************************** + * methods for step 2 + * ******************************************************************************************** */ + + /** + * required WindowFrame: (UNBOUNDED PRECEDING, offset PRECEDING) + * but in Spark, it is (offset PRECEDING, offset PRECEDING) + */ + @Override + public Lag visitLag(Lag lag, Void ctx) { + // check and complete window frame + windowExpression.getWindowFrame().ifPresent(wf -> { + throw new AnalysisException("WindowFrame for LAG() must be null"); + }); + if (lag.children().size() != 3) { + throw new AnalysisException("Lag must have three parameters"); + } + + Expression column = lag.child(0); + Expression offset = lag.getOffset(); + Expression defaultValue = lag.getDefaultValue(); + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newPrecedingBoundary(offset)); + windowExpression = windowExpression.withWindowFrame(requiredFrame); + + // check if the class of lag's column matches defaultValue, and cast it + if (!TypeCoercionUtils.implicitCast(column.getDataType(), defaultValue.getDataType()).isPresent()) { + throw new AnalysisException("DefaultValue's Datatype of LAG() cannot match its relevant column. The column " + + "type is " + column.getDataType() + ", but the defaultValue type is " + defaultValue.getDataType()); + } + return lag.withChildren(ImmutableList.of(column, offset, + TypeCoercionUtils.castIfNotSameType(defaultValue, column.getDataType()))); + } + + /** + * required WindowFrame: (UNBOUNDED PRECEDING, offset FOLLOWING) + * but in Spark, it is (offset FOLLOWING, offset FOLLOWING) + */ + @Override + public Lead visitLead(Lead lead, Void ctx) { + windowExpression.getWindowFrame().ifPresent(wf -> { + throw new AnalysisException("WindowFrame for LEAD() must be null"); + }); + if (lead.children().size() != 3) { + throw new AnalysisException("Lead must have three parameters"); + } + + Expression column = lead.child(0); + Expression offset = lead.getOffset(); + Expression defaultValue = lead.getDefaultValue(); + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newFollowingBoundary(offset)); + windowExpression = windowExpression.withWindowFrame(requiredFrame); + + // check if the class of lag's column matches defaultValue, and cast it + if (!TypeCoercionUtils.implicitCast(column.getDataType(), defaultValue.getDataType()).isPresent()) { + throw new AnalysisException("DefaultValue's Datatype of LEAD() can't match its relevant column. The column " + + "type is " + column.getDataType() + ", but the defaultValue type is " + defaultValue.getDataType()); + } + return lead.withChildren(ImmutableList.of(column, offset, + TypeCoercionUtils.castIfNotSameType(defaultValue, column.getDataType()))); + } + + /** + * [Copied from class AnalyticExpr.standardize()]: + * + * FIRST_VALUE without UNBOUNDED PRECEDING gets rewritten to use a different window + * and change the function to return the last value. We either set the fn to be + * 'last_value' or 'first_value_rewrite', which simply wraps the 'last_value' + * implementation but allows us to handle the first rows in a partition in a special + * way in the backend. There are a few cases: + * a) Start bound is X FOLLOWING or CURRENT ROW (X=0): + * Use 'last_value' with a window where both bounds are X FOLLOWING (or + * CURRENT ROW). Setting the start bound to X following is necessary because the + * X rows at the end of a partition have no rows in their window. Note that X + * FOLLOWING could be rewritten as lead(X) but that would not work for CURRENT + * ROW. + * b) Start bound is X PRECEDING and end bound is CURRENT ROW or FOLLOWING: + * Use 'first_value_rewrite' and a window with an end bound X PRECEDING. An + * extra parameter '-1' is added to indicate to the backend that NULLs should + * not be added for the first X rows. + * c) Start bound is X PRECEDING and end bound is Y PRECEDING: + * Use 'first_value_rewrite' and a window with an end bound X PRECEDING. The + * first Y rows in a partition have empty windows and should be NULL. An extra + * parameter with the integer constant Y is added to indicate to the backend + * that NULLs should be added for the first Y rows. + */ + @Override + public FirstOrLastValue visitFirstValue(FirstValue firstValue, Void ctx) { + Optional windowFrame = windowExpression.getWindowFrame(); + if (windowFrame.isPresent()) { + WindowFrame wf = windowFrame.get(); + if (wf.getLeftBoundary().isNot(FrameBoundType.UNBOUNDED_PRECEDING) + && wf.getLeftBoundary().isNot(FrameBoundType.PRECEDING)) { + windowExpression = windowExpression.withWindowFrame(wf.withRightBoundary(wf.getLeftBoundary())); + LastValue lastValue = new LastValue(firstValue.child()); + return lastValue; + } + + if (wf.getLeftBoundary().is(FrameBoundType.UNBOUNDED_PRECEDING) + && wf.getRightBoundary().isNot(FrameBoundType.PRECEDING)) { + windowExpression = windowExpression.withWindowFrame( + wf.withRightBoundary(FrameBoundary.newCurrentRowBoundary())); + } + } else { + windowExpression = windowExpression.withWindowFrame(new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary())); + } + return firstValue; + } + + /** + * required WindowFrame: (RANGE, UNBOUNDED PRECEDING, CURRENT ROW) + */ + @Override + public Rank visitRank(Rank rank, Void ctx) { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + + checkAndCompleteWindowFrame(requiredFrame, rank.getName()); + return rank; + } + + /** + * required WindowFrame: (RANGE, UNBOUNDED PRECEDING, CURRENT ROW) + */ + @Override + public DenseRank visitDenseRank(DenseRank denseRank, Void ctx) { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + + checkAndCompleteWindowFrame(requiredFrame, denseRank.getName()); + return denseRank; + } + + /** + * required WindowFrame: (ROWS, UNBOUNDED PRECEDING, CURRENT ROW) + */ + @Override + public RowNumber visitRowNumber(RowNumber rowNumber, Void ctx) { + // check and complete window frame + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + + checkAndCompleteWindowFrame(requiredFrame, rowNumber.getName()); + return rowNumber; + } + + /** + * required WindowFrame: (ROWS, UNBOUNDED PRECEDING, CURRENT ROW) + */ + @Override + public Ntile visitNtile(Ntile ntile, Void ctx) { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + + checkAndCompleteWindowFrame(requiredFrame, ntile.getName()); + return ntile; + } + + /** + * check if the current WindowFrame equals with the required WindowFrame; if current WindowFrame is null, + * the requiredFrame should be used as default frame. + */ + private void checkAndCompleteWindowFrame(WindowFrame requiredFrame, String functionName) { + windowExpression.getWindowFrame().ifPresent(wf -> { + if (!wf.equals(requiredFrame)) { + throw new AnalysisException("WindowFrame for " + functionName + "() must be null " + + "or match with " + requiredFrame); + } + }); + windowExpression = windowExpression.withWindowFrame(requiredFrame); + } + + /* ******************************************************************************************** + * methods for step 3 + * ******************************************************************************************** */ + + private void checkWindowFrameAfterFunc(WindowFrame wf) { + if (wf.getRightBoundary().is(FrameBoundType.UNBOUNDED_FOLLOWING) + && wf.getLeftBoundary().isNot(FrameBoundType.UNBOUNDED_PRECEDING)) { + // reverse OrderKey's asc and isNullFirst; + // in checkWindowFrameBeforeFunc(), we have confirmed that orderKeyLists must exist + List newOKList = windowExpression.getOrderKeys().stream() + .map(orderExpression -> { + OrderKey orderKey = orderExpression.getOrderKey(); + return new OrderExpression( + new OrderKey(orderKey.getExpr(), !orderKey.isAsc(), !orderKey.isNullFirst())); + }) + .collect(Collectors.toList()); + windowExpression = windowExpression.withOrderKeyList(newOKList); + + // reverse WindowFrame + // e.g. (3 preceding, unbounded following) -> (unbounded preceding, 3 following) + windowExpression = windowExpression.withWindowFrame(wf.reverseWindow()); + + // reverse WindowFunction, which is used only for first_value() and last_value() + Expression windowFunction = windowExpression.getFunction(); + if (windowFunction instanceof FirstOrLastValue) { + // windowExpression = windowExpression.withChildren( + // ImmutableList.of(((FirstOrLastValue) windowFunction).reverse())); + windowExpression = windowExpression.withFunction(((FirstOrLastValue) windowFunction).reverse()); + } + } + } + + private void setDefaultWindowFrameAfterFunc() { + // this is equal to DEFAULT_WINDOW in class AnalyticWindow + windowExpression = windowExpression.withWindowFrame(new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary())); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java new file mode 100644 index 00000000000000..55070438f764f8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java @@ -0,0 +1,545 @@ +// 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.implementation; + +import org.apache.doris.nereids.annotation.DependsRules; +import org.apache.doris.nereids.properties.DistributionSpecHash; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.properties.OrderSpec; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.properties.RequireProperties; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.logical.CheckAndStandardizeWindowFunctionAndFrame; +import org.apache.doris.nereids.rules.rewrite.logical.ExtractAndNormalizeWindowExpression; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Implementation rule that convert logical window to physical window, and add RequiredProperties + * + * step 1: compute three kinds of group: + * WindowFrameGroup: maintain windows with same PartitionKey, OrderKey and WindowFrame + * OrderKeyGroup: maintain windows with same PartitionKey and OrderKey + * PartitionKeyGroup: maintain windows with same PartitionKey + * step 2: sort PartitionKeyGroup with increasing order of tupleSize + * step 3: for every WindowFrameGroup of each SortGroup, generate one PhysicalWindow node, with common PartitionKeys, + * OrderKeys, unique WindowFrame and a function list. + * step 4: for each PhysicalWindow, generate RequiredProperties, including PartitionKey for DistributionSpec, + * and (PartitionKey + OrderKey) for OrderSpec. + */ +@DependsRules({ + CheckAndStandardizeWindowFunctionAndFrame.class, + ExtractAndNormalizeWindowExpression.class +}) +public class LogicalWindowToPhysicalWindow extends OneImplementationRuleFactory { + + @Override + public Rule build() { + + return RuleType.LOGICAL_WINDOW_TO_PHYSICAL_WINDOW_RULE.build( + logicalWindow().when(LogicalWindow::isChecked).then(this::implement) + ); + } + + /** + * main procedure + */ + private PhysicalWindow implement(LogicalWindow logicalWindow) { + // todo: remove windowExpressions from LogicalWindow and rule ExtractWindowExpressions. + // Only add this variable in PhysicalWindow + List windowList = logicalWindow.getWindowExpressions(); + + /////////// create three kinds of groups and compute tupleSize of each + // windowFrameGroup + List windowFrameGroupList = createWindowFrameGroups(windowList); + + // orderKeyGroup + List orderKeyGroupList = createOrderKeyGroups(windowFrameGroupList); + mergeOrderKeyGroups(orderKeyGroupList); + + // partitionKeyGroup + List partitionKeyGroupList = createPartitionKeyGroups(orderKeyGroupList); + // todo: complete the check of NumDistinctValues and open this function + // mergePartitionKeyGroups(partitionKeyGroupList); + + // sort groups by increasing tupleSize + sortPartitionKeyGroups(partitionKeyGroupList); + + // todo: optimize the order of partitionKeyGroups when there exist group keys below LogicalWindow + + Plan newRoot = logicalWindow.child(); + for (PartitionKeyGroup partitionKeyGroup : partitionKeyGroupList) { + for (OrderKeyGroup orderKeyGroup : partitionKeyGroup.groups) { + // in OrderKeyGroup, create PhysicalWindow for each WindowFrameGroup; + // each PhysicalWindow contains the same windowExpressions as WindowFrameGroup.groups + newRoot = createPhysicalPlanNodeForWindowFrameGroup(newRoot, orderKeyGroup); + } + } + return (PhysicalWindow) newRoot; + } + + /* ******************************************************************************************** + * create PhysicalWindow and PhysicalSort + * ******************************************************************************************** */ + + private Plan createPhysicalPlanNodeForWindowFrameGroup(Plan root, OrderKeyGroup orderKeyGroup) { + // PhysicalSort node for orderKeys; if there exists no orderKey, newRoot = root + // Plan newRoot = createPhysicalSortNode(root, orderKeyGroup, ctx); + Plan newRoot = root; + + // we will not add PhysicalSort in this step, but generate it if necessary with the ability of enforcer by + // setting RequiredProperties for PhysicalWindow + List requiredOrderKeys = generateKeysNeedToBeSorted(orderKeyGroup); + + // PhysicalWindow nodes for each different window frame, so at least one PhysicalWindow node will be added + for (WindowFrameGroup windowFrameGroup : orderKeyGroup.groups) { + newRoot = createPhysicalWindow(newRoot, windowFrameGroup, requiredOrderKeys); + } + + return newRoot; + } + + private List generateKeysNeedToBeSorted(OrderKeyGroup orderKeyGroup) { + // all keys that need to be sorted, which includes BOTH partitionKeys and orderKeys from this group + List keysNeedToBeSorted = Lists.newArrayList(); + + // used as SortNode.isAnalyticSort, but it is unnecessary to add it in LogicalSort + if (!orderKeyGroup.partitionKeys.isEmpty()) { + keysNeedToBeSorted.addAll(orderKeyGroup.partitionKeys.stream().map(partitionKey -> { + // todo: haven't support isNullFirst, and its default value is false(see AnalyticPlanner, + // but in LogicalPlanBuilder, its default value is true) + return new OrderKey(partitionKey, true, false); + }).collect(Collectors.toList())); + } + + if (!orderKeyGroup.orderKeys.isEmpty()) { + keysNeedToBeSorted.addAll(orderKeyGroup.orderKeys.stream() + .map(orderExpression -> orderExpression.getOrderKey()) + .collect(Collectors.toList()) + ); + } + return keysNeedToBeSorted; + } + + private PhysicalWindow createPhysicalWindow(Plan root, WindowFrameGroup windowFrameGroup, + List requiredOrderKeys) { + // requiredProperties: + // Distribution: partitionKeys + // Order: requiredOrderKeys + LogicalWindow tempLogicalWindow = new LogicalWindow<>(windowFrameGroup.groups, root); + PhysicalWindow physicalWindow = new PhysicalWindow<>( + windowFrameGroup, + RequireProperties.followParent(), + tempLogicalWindow.getLogicalProperties(), + root); + + if (windowFrameGroup.partitionKeys.isEmpty() && requiredOrderKeys.isEmpty()) { + return physicalWindow; + } + + // todo: WFGs in the same OKG only need same RequiredProperties + PhysicalProperties properties; + if (windowFrameGroup.partitionKeys.isEmpty()) { + properties = new PhysicalProperties(new OrderSpec(requiredOrderKeys)); + } else { + properties = PhysicalProperties.createHash( + windowFrameGroup.partitionKeys, DistributionSpecHash.ShuffleType.ENFORCED); + // requiredOrderKeys contain partitionKeys, so there is no need to check if requiredOrderKeys.isEmpty() + properties = properties.withOrderSpec(new OrderSpec(requiredOrderKeys)); + } + + RequireProperties requireProperties = RequireProperties.of(properties); + return physicalWindow.withRequirePropertiesAndChild(requireProperties, root); + } + + /* ******************************************************************************************** + * WindowFunctionRelatedGroups + * ******************************************************************************************** */ + + // todo: can we simplify the following three algorithms? + private List createWindowFrameGroups(List windowList) { + List windowFrameGroupList = Lists.newArrayList(); + for (int i = 0; i < windowList.size(); i++) { + NamedExpression windowAlias = windowList.get(i); + + boolean matched = false; + for (WindowFrameGroup windowFrameGroup : windowFrameGroupList) { + if (windowFrameGroup.isCompatible(windowAlias)) { + windowFrameGroup.addGroup(windowAlias); + matched = true; + break; + } + } + if (!matched) { + windowFrameGroupList.add(new WindowFrameGroup(windowAlias)); + } + } + + for (WindowFrameGroup wfg : windowFrameGroupList) { + wfg.setTupleSize(wfg.groups.stream() + .mapToInt(window -> window.child(0).getDataType().width()) + .sum()); + } + + return windowFrameGroupList; + } + + private List createOrderKeyGroups(List windowFrameGroupList) { + List orderKeyGroupList = Lists.newArrayList(); + + for (WindowFrameGroup windowFrameGroup : windowFrameGroupList) { + boolean matched = false; + for (OrderKeyGroup orderKeyGroup : orderKeyGroupList) { + // looks unreadable. An OrderKeyGroup contains a list of WFGs having same partitionKeys and orderKeys. + // An OKG isCompatible an WFG means this WFG also has same keys. + if (orderKeyGroup.isCompatible(windowFrameGroup)) { + orderKeyGroup.addGroup(windowFrameGroup); + matched = true; + break; + } + } + if (!matched) { + orderKeyGroupList.add(new OrderKeyGroup(windowFrameGroup)); + } + } + + for (OrderKeyGroup okg : orderKeyGroupList) { + okg.setTupleSize(okg.getGroups().stream() + .mapToInt(WindowFrameGroup::getTupleSize) + .sum()); + } + + return orderKeyGroupList; + } + + private List createPartitionKeyGroups(List orderKeyGroupList) { + List partitionKeyGroupList = Lists.newArrayList(); + + for (OrderKeyGroup orderKeyGroup : orderKeyGroupList) { + boolean matched = false; + for (PartitionKeyGroup partitionKeyGroup : partitionKeyGroupList) { + if (partitionKeyGroup.isCompatible(orderKeyGroup)) { + partitionKeyGroup.addGroup(orderKeyGroup); + matched = true; + break; + } + } + if (!matched) { + partitionKeyGroupList.add(new PartitionKeyGroup(orderKeyGroup)); + } + } + + for (PartitionKeyGroup pkg : partitionKeyGroupList) { + pkg.setTupleSize(pkg.getGroups().stream() + .mapToInt(OrderKeyGroup::getTupleSize) + .sum()); + } + + return partitionKeyGroupList; + } + + private void mergeOrderKeyGroups(List orderKeyGroupList) { + boolean merged = true; + + while (merged) { + merged = false; + for (OrderKeyGroup okg1 : orderKeyGroupList) { + for (OrderKeyGroup okg2 : orderKeyGroupList) { + if (okg1 != okg2 && okg2.isPrefixOf(okg1)) { + // okg2 ∈ okg1, so we remove okg2, and add all the WFG in okg2 to okg1 + okg1.absorb(okg2); + orderKeyGroupList.remove(okg2); + merged = true; + break; + } + } + if (merged) { + break; + } + } + } + } + + private void mergePartitionKeyGroups(List partitionKeyGroupList) { + boolean merged = true; + + while (merged) { + merged = false; + for (PartitionKeyGroup pkg1 : partitionKeyGroupList) { + for (PartitionKeyGroup pkg2 : partitionKeyGroupList) { + if (pkg1 != pkg2) { + // todo: check NumDistinctValues like legacy planner + pkg1.absorb(pkg2); + partitionKeyGroupList.remove(pkg2); + merged = true; + break; + } + } + if (merged) { + break; + } + } + } + } + + /** + * [This comment references AnalyticPlanner.orderGroups()] + * + * Order partition groups (and the SortGroups, WindowFrameGroups within them) by increasing + * totalOutputTupleSize. This minimizes the total volume of data that needs to be + * repartitioned and sorted. + * + * Always move the non-partitioning partition group to the end, if it exists. + */ + private void sortPartitionKeyGroups(List partitionKeyGroupList) { + + PartitionKeyGroup noPartition = null; + // after createPartitionKeyGroups(), there will be at most one empty partition. + for (PartitionKeyGroup pkg : partitionKeyGroupList) { + if (pkg.partitionKeys.isEmpty()) { + noPartition = pkg; + partitionKeyGroupList.remove(noPartition); + break; + } + } + + partitionKeyGroupList.sort((pkg1, pkg2) -> + Integer.compare(pkg1.getTupleSize() - pkg2.getTupleSize(), 0) + ); + + if (noPartition != null) { + partitionKeyGroupList.add(noPartition); + } + + for (PartitionKeyGroup pkg : partitionKeyGroupList) { + sortOrderKeyGroups(pkg.getGroups()); + } + + } + + private void sortOrderKeyGroups(List orderKeyGroupList) { + orderKeyGroupList.sort((okg1, okg2) -> + Integer.compare(okg1.getTupleSize() - okg2.getTupleSize(), 0) + ); + + for (OrderKeyGroup okg : orderKeyGroupList) { + sortWindowFrameGroups(okg.getGroups()); + } + } + + private void sortWindowFrameGroups(List windowFrameGroupList) { + windowFrameGroupList.sort((wfg1, wfg2) -> + Integer.compare(wfg1.getTupleSize() - wfg2.getTupleSize(), 0) + ); + } + + /** + * Window Functions that have common PartitionKeys, OrderKeys and WindowFrame + */ + public static class WindowFrameGroup extends WindowFunctionRelatedGroup { + + private final Set partitionKeys; + private final List orderKeys; + private final WindowFrame windowFrame; + + public WindowFrameGroup(NamedExpression windowAlias) { + WindowExpression window = (WindowExpression) (windowAlias.child(0)); + partitionKeys = ImmutableSet.copyOf(window.getPartitionKeys()); + orderKeys = window.getOrderKeys(); + windowFrame = window.getWindowFrame().get(); + groups.add(windowAlias); + } + + @Override + public void addGroup(NamedExpression windowAlias) { + groups.add(windowAlias); + } + + @Override + public boolean isCompatible(NamedExpression windowAlias) { + // The comparison of PartitionKey is irrelevant to key's order, + // but not in OrderKey' comparison. + WindowExpression window = (WindowExpression) (windowAlias.child(0)); + + Set otherPartitionKeys = ImmutableSet.copyOf(window.getPartitionKeys()); + List otherOrderKeys = window.getOrderKeys(); + // WindowFrame will not be null after standardizing + WindowFrame otherWindowFrame = window.getWindowFrame().get(); + + // for PartitionKeys, we don't care about the order of each key, so we use isEqualCollection() to compare + // whether these two lists have same keys; but for OrderKeys, the order of each key also make sense, so + // we use equals() to compare both the elements and their order in these two lists. + + // CollectionUtils.isEqualCollection() is absolutely equals to Expr.equalSets() + return CollectionUtils.isEqualCollection(partitionKeys, otherPartitionKeys) + && orderKeys.equals(otherOrderKeys) + && windowFrame.equals(otherWindowFrame); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("(Funcs=").append(groups.stream() + .map(Expression::toString) + .collect(Collectors.joining(", ", "[", "], "))); + sb.append("PartitionKeys=").append(partitionKeys.stream() + .map(Expression::toString) + .collect(Collectors.joining(", ", "[", "], "))); + sb.append("OrderKeys=").append(orderKeys.stream() + .map(OrderExpression::toString) + .collect(Collectors.joining(", ", "[", "], "))); + sb.append("WindowFrame=").append(windowFrame); + return sb + ")"; + } + + public Set getPartitionKeys() { + return partitionKeys; + } + + public List getOrderKeys() { + return orderKeys; + } + + public WindowFrame getWindowFrame() { + return windowFrame; + } + + } + + /** + * Window Functions that have common PartitionKeys and OrderKeys. + */ + private static class OrderKeyGroup extends WindowFunctionRelatedGroup { + + private final Set partitionKeys; + private final List orderKeys; + + public OrderKeyGroup(WindowFrameGroup windowFrameGroup) { + partitionKeys = windowFrameGroup.partitionKeys; + orderKeys = windowFrameGroup.orderKeys; + groups.add(windowFrameGroup); + } + + @Override + public boolean isCompatible(WindowFrameGroup windowFrameGroup) { + return CollectionUtils.isEqualCollection(partitionKeys, windowFrameGroup.partitionKeys) + && orderKeys.equals(windowFrameGroup.orderKeys); + } + + /** + * check if this okg isPrefixOf other, which means that other okg can cover this + * e.g. (a, b) is a prefix of (a, b, c), but is NOT a prefix of (b, a, c) or (b, c) + */ + public boolean isPrefixOf(OrderKeyGroup otherOkg) { + if (orderKeys.size() > otherOkg.orderKeys.size()) { + return false; + } + if (!CollectionUtils.isEqualCollection(partitionKeys, otherOkg.partitionKeys)) { + return false; + } + + for (int i = 0; i < orderKeys.size(); i++) { + if (!orderKeys.get(i).equals(otherOkg.orderKeys.get(i))) { + return false; + } + } + return true; + } + + /** + * add all of otherOkg's WindowFrameGroups to this groupList + */ + public void absorb(OrderKeyGroup otherOkg) { + groups.addAll(otherOkg.groups); + tupleSize += (otherOkg.groups.stream() + .mapToInt(WindowFrameGroup::getTupleSize) + .sum()); + } + + } + + /** + * Window Functions that have common PartitionKeys. + */ + private static class PartitionKeyGroup extends WindowFunctionRelatedGroup { + public Set partitionKeys; + + public PartitionKeyGroup(OrderKeyGroup orderKeyGroup) { + partitionKeys = orderKeyGroup.partitionKeys; + groups.add(orderKeyGroup); + } + + @Override + public boolean isCompatible(OrderKeyGroup orderKeyGroup) { + return CollectionUtils.isEqualCollection(partitionKeys, orderKeyGroup.partitionKeys); + } + + /** + * absorb other into this: + * - partitionKeyList will be the intersection of this two group + * - groupList(orderKeyGroup) will be the union of this two group + */ + public void absorb(PartitionKeyGroup otherPkg) { + partitionKeys = partitionKeys.stream() + .filter(expression -> otherPkg.partitionKeys.contains(expression)) + .collect(ImmutableSet.toImmutableSet()); + groups.addAll(otherPkg.groups); + } + } + + private abstract static class WindowFunctionRelatedGroup { + + protected List groups = Lists.newArrayList(); + + protected int tupleSize; + + public abstract boolean isCompatible(G group); + + public void addGroup(G group) { + groups.add(group); + } + + public List getGroups() { + return groups; + } + + public int getTupleSize() { + return tupleSize; + } + + public void setTupleSize(int tupleSize) { + this.tupleSize = tupleSize; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionAndFrame.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionAndFrame.java new file mode 100644 index 00000000000000..2855f6376f94b0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionAndFrame.java @@ -0,0 +1,65 @@ +// 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.logical; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.analysis.WindowFunctionChecker; +import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.plans.GroupPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Check and standardize Window expression + */ +public class CheckAndStandardizeWindowFunctionAndFrame extends OneRewriteRuleFactory { + + @Override + public Rule build() { + return RuleType.CHECK_AND_STANDARDIZE_WINDOW_FUNCTION_AND_FRAME.build( + logicalWindow().whenNot(LogicalWindow::isChecked).then(logicalWindow -> + checkAndStandardize(logicalWindow)) + ); + } + + private LogicalWindow checkAndStandardize(LogicalWindow logicalWindow) { + + List newOutputExpressions = logicalWindow.getWindowExpressions().stream() + .map(expr -> { + WindowExpression window = (WindowExpression) expr.child(0); + WindowFunctionChecker checker = new WindowFunctionChecker(window); + checker.checkWindowBeforeFunc(); + checker.checkWindowFunction(); + checker.checkWindowAfterFunc(); + WindowExpression newWindow = checker.getWindow(); + Preconditions.checkArgument(newWindow.getWindowFrame().isPresent(), + "WindowFrame shouldn't be null after checkAndStandardize"); + return (Alias) expr.withChildren(newWindow); + }) + .collect(Collectors.toList()); + return logicalWindow.withChecked(newOutputExpressions, logicalWindow.child()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpression.java new file mode 100644 index 00000000000000..54121ad5b62357 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpression.java @@ -0,0 +1,117 @@ +// 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.logical; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; +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.WindowExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Set; + +/** + * extract window expressions from LogicalProject.projects and Normalize LogicalWindow + */ +public class ExtractAndNormalizeWindowExpression extends OneRewriteRuleFactory implements NormalizeToSlot { + + @Override + public Rule build() { + return logicalProject().when(project -> containsWindowExpression(project.getProjects())).then(project -> { + List outputs = project.getProjects(); + + // 1. handle bottom projects + Set existedAlias = ExpressionUtils.collect(outputs, Alias.class::isInstance); + Set toBePushedDown = collectExpressionsToBePushedDown(outputs); + NormalizeToSlotContext context = NormalizeToSlotContext.buildContext(existedAlias, toBePushedDown); + // set toBePushedDown exprs as NamedExpression, e.g. (a+1) -> Alias(a+1) + Set bottomProjects = context.pushDownToNamedExpression(toBePushedDown); + Plan normalizedChild; + if (bottomProjects.isEmpty()) { + normalizedChild = project.child(); + } else { + boolean needAggregate = bottomProjects.stream().anyMatch(expr -> + expr.anyMatch(AggregateFunction.class::isInstance)); + if (needAggregate) { + normalizedChild = new LogicalAggregate<>( + ImmutableList.of(), ImmutableList.copyOf(bottomProjects), project.child()); + } else { + normalizedChild = new LogicalProject<>(ImmutableList.copyOf(bottomProjects), project.child()); + } + } + + // 2. handle window's outputs and windowExprs + // need to replace exprs with SlotReference in WindowSpec, due to LogicalWindow.getExpressions() + List normalizedOutputs1 = context.normalizeToUseSlotRef(outputs); + Set normalizedWindows = + ExpressionUtils.collect(normalizedOutputs1, WindowExpression.class::isInstance); + + existedAlias = ExpressionUtils.collect(normalizedOutputs1, Alias.class::isInstance); + NormalizeToSlotContext ctxForWindows = NormalizeToSlotContext.buildContext( + existedAlias, Sets.newHashSet(normalizedWindows)); + + Set normalizedWindowWithAlias = ctxForWindows.pushDownToNamedExpression(normalizedWindows); + // only need normalized windowExpressions + LogicalWindow normalizedLogicalWindow = + new LogicalWindow(Lists.newArrayList(normalizedWindowWithAlias), normalizedChild); + + // 3. handle top projects + List topProjects = ctxForWindows.normalizeToUseSlotRef(normalizedOutputs1); + return new LogicalProject<>(topProjects, normalizedLogicalWindow); + }).toRule(RuleType.EXTRACT_AND_NORMALIZE_WINDOW_EXPRESSIONS); + } + + private Set collectExpressionsToBePushedDown(List expressions) { + // bottomProjects includes: + // 1. expressions from function and WindowSpec's partitionKeys and orderKeys + // 2. other slots of outputExpressions + return expressions.stream() + .flatMap(expression -> { + if (expression.anyMatch(WindowExpression.class::isInstance)) { + Set collects = expression.collect(WindowExpression.class::isInstance); + return collects.stream().flatMap(windowExpression -> + windowExpression.getExpressionsInWindowSpec().stream() + // constant arguments may in WindowFunctions(e.g. Lead, Lag), which shouldn't be pushed down + .filter(expr -> !expr.isConstant()) + ); + } + return ImmutableList.of(expression).stream(); + }) + .collect(ImmutableSet.toImmutableSet()); + } + + private boolean containsWindowExpression(List expressions) { + // WindowExpression in top LogicalProject will be normalized as Alias(SlotReference) after this rule, + // so it will not be normalized infinitely + return expressions.stream().anyMatch(expr -> expr.anyMatch(WindowExpression.class::isInstance)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeAggregate.java index e97fc993bac97c..07ddffeced698e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeAggregate.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; @@ -36,6 +37,7 @@ import java.util.List; import java.util.Set; +import java.util.stream.Collectors; /** * normalize aggregate's group keys and AggregateFunction's child to SlotReference @@ -78,10 +80,16 @@ public Rule build() { // replace groupBy and arguments of aggregate function to slot, may be this output contains // some expression on the aggregate functions, e.g. `sum(value) + 1`, we should replace // the sum(value) to slot and move the `slot + 1` to the upper project later. - List normalizeOutputPhase1 = groupByAndArgumentToSlotContext - .normalizeToUseSlotRef(aggregate.getOutputExpressions()); + List normalizeOutputPhase1 = aggregate.getOutputExpressions().stream() + .map(expr -> { + if (expr.anyMatch(WindowExpression.class::isInstance)) { + return expr; + } + return groupByAndArgumentToSlotContext.normalizeToUseSlotRef(expr); + }).collect(Collectors.toList()); + Set normalizedAggregateFunctions = - ExpressionUtils.collect(normalizeOutputPhase1, AggregateFunction.class::isInstance); + collectNonWindowedAggregateFunctions(normalizeOutputPhase1); existsAliases = ExpressionUtils.collect(normalizeOutputPhase1, Alias.class::isInstance); @@ -107,9 +115,14 @@ public Rule build() { LogicalAggregate normalizedAggregate = aggregate.withNormalized( (List) normalizedGroupBy, normalizedAggregateOutput, normalizedChild); - // replace aggregate function to slot - List upperProjects = - aggregateFunctionToSlotContext.normalizeToUseSlotRef(normalizeOutputPhase1); + // exclude same-name functions in WindowExpression + List upperProjects = normalizeOutputPhase1.stream() + .map(expr -> { + if (expr.anyMatch(WindowExpression.class::isInstance)) { + return expr; + } + return aggregateFunctionToSlotContext.normalizeToUseSlotRef(expr); + }).collect(Collectors.toList()); return new LogicalProject<>(upperProjects, normalizedAggregate); }).toRule(RuleType.NORMALIZE_AGGREGATE); } @@ -120,8 +133,8 @@ private Set collectGroupByAndArgumentsOfAggregateFunctions(LogicalAg Set groupingByExpr = ImmutableSet.copyOf(aggregate.getGroupByExpressions()); - Set aggregateFunctions = ExpressionUtils.collect( - aggregate.getOutputExpressions(), AggregateFunction.class::isInstance); + Set aggregateFunctions = collectNonWindowedAggregateFunctions( + aggregate.getOutputExpressions()); ImmutableSet argumentsOfAggregateFunction = aggregateFunctions.stream() .flatMap(function -> function.getArguments().stream().map(arg -> { @@ -142,4 +155,12 @@ private Set collectGroupByAndArgumentsOfAggregateFunctions(LogicalAg .build(); return needPushDown; } + + private Set collectNonWindowedAggregateFunctions(List aggOutput) { + List expressionsWithoutWindow = aggOutput.stream() + .filter(expr -> !expr.anyMatch(WindowExpression.class::isInstance)) + .collect(Collectors.toList()); + + return ExpressionUtils.collect(expressionsWithoutWindow, AggregateFunction.class::isInstance); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeToSlot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeToSlot.java index 34686534e39548..aee929c8beeae2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeToSlot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/NormalizeToSlot.java @@ -65,6 +65,10 @@ public static NormalizeToSlotContext buildContext( return new NormalizeToSlotContext(normalizeToSlotMap); } + public E normalizeToUseSlotRef(E expression) { + return normalizeToUseSlotRef(ImmutableList.of(expression)).get(0); + } + /** normalizeToUseSlotRef, no custom normalize */ public List normalizeToUseSlotRef(List expressions) { return normalizeToUseSlotRef(expressions, (context, expr) -> expr); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index e6ae621ca8b59a..a53f1ef82fadf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -38,6 +38,7 @@ import org.apache.doris.nereids.trees.plans.algebra.Scan; import org.apache.doris.nereids.trees.plans.algebra.SetOperation; import org.apache.doris.nereids.trees.plans.algebra.TopN; +import org.apache.doris.nereids.trees.plans.algebra.Window; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; @@ -57,6 +58,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; @@ -80,6 +82,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.ColumnStatisticBuilder; @@ -234,6 +237,15 @@ public StatsDeriveResult visitLogicalGenerate(LogicalGenerate ge return computeGenerate(generate); } + public StatsDeriveResult visitLogicalWindow(LogicalWindow window, Void context) { + return computeWindow(window); + } + + @Override + public StatsDeriveResult visitPhysicalWindow(PhysicalWindow window, Void context) { + return computeWindow(window); + } + @Override public StatsDeriveResult visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, Void context) { return computeEmptyRelation(emptyRelation); @@ -611,4 +623,30 @@ private StatsDeriveResult computeGenerate(Generate generate) { } return new StatsDeriveResult(count, columnStatsMap); } + + private StatsDeriveResult computeWindow(Window windowOperator) { + StatsDeriveResult stats = groupExpression.childStatistics(0); + Map childColumnStats = stats.getSlotIdToColumnStats(); + Map columnStatisticMap = windowOperator.getWindowExpressions().stream() + .map(expr -> { + ColumnStatistic value = null; + Set slots = expr.getInputSlots(); + if (slots.isEmpty()) { + value = ColumnStatistic.DEFAULT; + } else { + for (Slot slot : slots) { + if (childColumnStats.containsKey(slot.getExprId())) { + value = childColumnStats.get(slot.getExprId()); + break; + } + } + if (value == null) { + // todo: how to set stats? + value = ColumnStatistic.DEFAULT; + } + } + return Pair.of(expr.toSlot().getExprId(), value); + }).collect(Collectors.toMap(Pair::key, Pair::value)); + return new StatsDeriveResult(stats.getRowCount(), columnStatisticMap); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java new file mode 100644 index 00000000000000..c9e96da1ec14c0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowExpression.java @@ -0,0 +1,215 @@ +// 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; + +import org.apache.doris.nereids.exceptions.UnboundException; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * represents window function. WindowFunction of this window is saved as Window's child, + * which is an UnboundFunction at first and will be analyzed as relevant BoundFunction + * (can be a WindowFunction or AggregateFunction) after BindFunction. + */ +public class WindowExpression extends Expression { + + private final Expression function; + + private final List partitionKeys; + + private final List orderKeys; + + private final Optional windowFrame; + + /** constructor of Window*/ + public WindowExpression(Expression function, List partitionKeys, List orderKeys) { + super(new ImmutableList.Builder() + .add(function) + .addAll(partitionKeys) + .addAll(orderKeys) + .build().toArray(new Expression[0])); + this.function = function; + this.partitionKeys = ImmutableList.copyOf(partitionKeys); + this.orderKeys = ImmutableList.copyOf(orderKeys); + this.windowFrame = Optional.empty(); + } + + /** constructor of Window*/ + public WindowExpression(Expression function, List partitionKeys, List orderKeys, + WindowFrame windowFrame) { + super(new ImmutableList.Builder() + .add(function) + .addAll(partitionKeys) + .addAll(orderKeys) + .add(windowFrame) + .build().toArray(new Expression[0])); + this.function = function; + this.partitionKeys = ImmutableList.copyOf(partitionKeys); + this.orderKeys = ImmutableList.copyOf(orderKeys); + this.windowFrame = Optional.of(Objects.requireNonNull(windowFrame)); + } + + public Expression getFunction() { + return function; + } + + /** + * extract expressions from function, partitionKeys and orderKeys + * todo: expressions from WindowFrame + */ + public List getExpressionsInWindowSpec() { + List expressions = Lists.newArrayList(); + expressions.addAll(function.children()); + expressions.addAll(partitionKeys); + expressions.addAll(orderKeys.stream() + .map(orderExpression -> orderExpression.child()) + .collect(Collectors.toList())); + return expressions; + } + + public List getPartitionKeys() { + return partitionKeys; + } + + public List getOrderKeys() { + return orderKeys; + } + + public Optional getWindowFrame() { + return windowFrame; + } + + public WindowExpression withWindowFrame(WindowFrame windowFrame) { + return new WindowExpression(function, partitionKeys, orderKeys, windowFrame); + } + + public WindowExpression withOrderKeyList(List orderKeyList) { + if (windowFrame.isPresent()) { + return new WindowExpression(function, partitionKeys, orderKeyList, windowFrame.get()); + } + return new WindowExpression(function, partitionKeys, orderKeyList); + } + + public WindowExpression withFunction(Expression function) { + if (windowFrame.isPresent()) { + return new WindowExpression(function, partitionKeys, orderKeys, windowFrame.get()); + } + return new WindowExpression(function, partitionKeys, orderKeys); + } + + @Override + public boolean nullable() { + return function.nullable(); + } + + @Override + public WindowExpression withChildren(List children) { + Preconditions.checkArgument(children.size() >= 1); + int index = 0; + Expression func = children.get(index); + index += 1; + + List partitionKeys = children.subList(index, index + this.partitionKeys.size()); + index += this.partitionKeys.size(); + + List orderKeys = children.subList(index, index + this.orderKeys.size()).stream() + .map(OrderExpression.class::cast) + .collect(Collectors.toList()); + index += this.orderKeys.size(); + + if (index < children.size()) { + return new WindowExpression(func, partitionKeys, orderKeys, (WindowFrame) children.get(index)); + } + return new WindowExpression(func, partitionKeys, orderKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WindowExpression window = (WindowExpression) o; + return Objects.equals(function, window.function) + && Objects.equals(partitionKeys, window.partitionKeys) + && Objects.equals(orderKeys, window.orderKeys) + && Objects.equals(windowFrame, window.windowFrame); + } + + @Override + public int hashCode() { + return Objects.hash(function, partitionKeys, orderKeys, windowFrame); + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append(function.toSql() + " OVER("); + if (!partitionKeys.isEmpty()) { + sb.append("PARTITION BY ").append(partitionKeys.stream() + .map(Expression::toSql) + .collect(Collectors.joining(", ", "", " "))); + } + if (!orderKeys.isEmpty()) { + sb.append("ORDER BY ").append(orderKeys.stream() + .map(OrderExpression::toSql) + .collect(Collectors.joining(", ", "", " "))); + } + windowFrame.ifPresent(wf -> sb.append(wf.toSql())); + return sb.toString().trim() + ")"; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(function + " WindowSpec("); + if (!partitionKeys.isEmpty()) { + sb.append("PARTITION BY ").append(partitionKeys.stream() + .map(Expression::toString) + .collect(Collectors.joining(", ", "", " "))); + } + if (!orderKeys.isEmpty()) { + sb.append("ORDER BY ").append(orderKeys.stream() + .map(OrderExpression::toString) + .collect(Collectors.joining(", ", "", " "))); + } + windowFrame.ifPresent(wf -> sb.append(wf.toSql())); + return sb.toString().trim() + ")"; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitWindow(this, context); + } + + @Override + public DataType getDataType() throws UnboundException { + return function.getDataType(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java new file mode 100644 index 00000000000000..121562cebb4338 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/WindowFrame.java @@ -0,0 +1,270 @@ +// 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; + +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; + +import java.util.Objects; +import java.util.Optional; + +/** + * window frame + */ +public class WindowFrame extends Expression implements PropagateNullable, LeafExpression { + + private final FrameUnitsType frameUnits; + + private final FrameBoundary leftBoundary; + + private final FrameBoundary rightBoundary; + + public WindowFrame(FrameUnitsType frameUnits, FrameBoundary leftBoundary) { + this(frameUnits, leftBoundary, new FrameBoundary(FrameBoundType.EMPTY_BOUNDARY)); + } + + public WindowFrame(FrameUnitsType frameUnits, FrameBoundary leftBoundary, FrameBoundary rightBoundary) { + this.frameUnits = frameUnits; + this.leftBoundary = leftBoundary; + this.rightBoundary = rightBoundary; + } + + public FrameUnitsType getFrameUnits() { + return frameUnits; + } + + public FrameBoundary getLeftBoundary() { + return leftBoundary; + } + + public FrameBoundary getRightBoundary() { + return rightBoundary; + } + + /** + * reverse left & right boundary; reverse each boundary's upper and lower bound + */ + public WindowFrame reverseWindow() { + return new WindowFrame(frameUnits, rightBoundary.reverse(), leftBoundary.reverse()); + } + + public WindowFrame withRightBoundary(FrameBoundary newRightBoundary) { + return new WindowFrame(frameUnits, leftBoundary, newRightBoundary); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WindowFrame other = (WindowFrame) o; + return Objects.equals(this.frameUnits, other.frameUnits) + && Objects.equals(this.leftBoundary, other.leftBoundary) + && Objects.equals(this.rightBoundary, other.rightBoundary); + } + + @Override + public int hashCode() { + return Objects.hash(frameUnits, leftBoundary, rightBoundary); + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append(frameUnits + " "); + if (rightBoundary != null) { + sb.append("BETWEEN " + leftBoundary + " AND " + rightBoundary); + } else { + sb.append(leftBoundary); + } + return sb.toString(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("WindowFrame("); + sb.append(frameUnits + ", "); + sb.append(leftBoundary + ", "); + if (rightBoundary != null) { + sb.append(rightBoundary); + } + sb.append(")"); + return sb.toString(); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitWindowFrame(this, context); + } + + /** + * frame units types + */ + public enum FrameUnitsType { + ROWS, + RANGE + } + + /** + * frame boundary + */ + public static class FrameBoundary { + + private final Optional boundOffset; + private final FrameBoundType frameBoundType; + + public FrameBoundary(FrameBoundType frameBoundType) { + this.boundOffset = Optional.empty(); + this.frameBoundType = frameBoundType; + } + + public FrameBoundary(Optional boundOffset, FrameBoundType frameBoundType) { + this.boundOffset = boundOffset; + this.frameBoundType = frameBoundType; + } + + public static FrameBoundary newPrecedingBoundary() { + return new FrameBoundary(FrameBoundType.UNBOUNDED_PRECEDING); + } + + public static FrameBoundary newPrecedingBoundary(Expression boundValue) { + return new FrameBoundary(Optional.of(boundValue), FrameBoundType.PRECEDING); + } + + public static FrameBoundary newFollowingBoundary() { + return new FrameBoundary(FrameBoundType.UNBOUNDED_FOLLOWING); + } + + public static FrameBoundary newFollowingBoundary(Expression boundValue) { + return new FrameBoundary(Optional.of(boundValue), FrameBoundType.FOLLOWING); + } + + public static FrameBoundary newCurrentRowBoundary() { + return new FrameBoundary(FrameBoundType.CURRENT_ROW); + } + + public boolean is(FrameBoundType otherType) { + return this.frameBoundType == otherType; + } + + public boolean isNot(FrameBoundType otherType) { + return this.frameBoundType != otherType; + } + + public boolean isNull() { + return this.frameBoundType == FrameBoundType.EMPTY_BOUNDARY; + } + + public boolean hasOffset() { + return frameBoundType == FrameBoundType.PRECEDING || frameBoundType == FrameBoundType.FOLLOWING; + } + + public boolean asPreceding() { + return frameBoundType == FrameBoundType.PRECEDING || frameBoundType == FrameBoundType.UNBOUNDED_PRECEDING; + } + + public boolean asFollowing() { + return frameBoundType == FrameBoundType.FOLLOWING || frameBoundType == FrameBoundType.UNBOUNDED_FOLLOWING; + } + + public FrameBoundary reverse() { + return new FrameBoundary(boundOffset, frameBoundType.reverse()); + } + + public FrameBoundType getFrameBoundType() { + return frameBoundType; + } + + public Optional getBoundOffset() { + return boundOffset; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + boundOffset.ifPresent(value -> sb.append(value + " ")); + sb.append(frameBoundType); + + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FrameBoundary other = (FrameBoundary) o; + return Objects.equals(this.frameBoundType, other.frameBoundType) + && Objects.equals(this.boundOffset, other.boundOffset); + } + + @Override + public int hashCode() { + return Objects.hash(boundOffset, frameBoundType); + } + } + + /** + * frame bound types + */ + public enum FrameBoundType { + + UNBOUNDED_PRECEDING, + UNBOUNDED_FOLLOWING, + CURRENT_ROW, + PRECEDING, + FOLLOWING, + + // represents that the boundary is null. We use this value as default + // to avoid checking if a boundary is null frequently. + EMPTY_BOUNDARY; + + /** + * reverse current FrameBoundType + */ + public FrameBoundType reverse() { + switch (this) { + case UNBOUNDED_PRECEDING: + return UNBOUNDED_FOLLOWING; + case UNBOUNDED_FOLLOWING: + return UNBOUNDED_PRECEDING; + case PRECEDING: + return FOLLOWING; + case FOLLOWING: + return PRECEDING; + case CURRENT_ROW: + return CURRENT_ROW; + default: + return EMPTY_BOUNDARY; + } + } + + public boolean isFollowing() { + return this.equals(UNBOUNDED_FOLLOWING) || this.equals(FOLLOWING); + } + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/DenseRank.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/DenseRank.java new file mode 100644 index 00000000000000..cc50ee5bfd37ba --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/DenseRank.java @@ -0,0 +1,61 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Window function: Dense_rank() + */ +public class DenseRank extends WindowFunction implements AlwaysNotNullable, LeafExpression { + + public DenseRank() { + super("dense_rank"); + } + + @Override + public List getSignatures() { + return ImmutableList.of(FunctionSignature.ret(BigIntType.INSTANCE).args()); + } + + @Override + public FunctionSignature searchSignature(List signatures) { + return signatures.get(0); + } + + @Override + public DenseRank withChildren(List children) { + Preconditions.checkArgument(children.size() == 0); + return new DenseRank(); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitDenseRank(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstOrLastValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstOrLastValue.java new file mode 100644 index 00000000000000..26e20c76d968a4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstOrLastValue.java @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; +import org.apache.doris.nereids.types.DataType; + +/** parent class for first_value() and last_value() */ +public abstract class FirstOrLastValue extends WindowFunction + implements UnaryExpression, PropagateNullable, CustomSignature { + + public FirstOrLastValue(String name, Expression child) { + super(name, child); + } + + public FirstOrLastValue reverse() { + if (this instanceof FirstValue) { + return new LastValue(child()); + } else { + return new FirstValue(child()); + } + } + + @Override + public FunctionSignature customSignature() { + DataType dataType = getArgument(0).getDataType(); + return FunctionSignature.ret(dataType).args(dataType); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstValue.java new file mode 100644 index 00000000000000..834f2f3a65a6d4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/FirstValue.java @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.window; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; + +import com.google.common.base.Preconditions; + +import java.util.List; + +/** + * Window function: First_value() + */ +public class FirstValue extends FirstOrLastValue { + + public FirstValue(Expression child) { + super("first_value", child); + } + + @Override + public FirstValue withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new FirstValue(children.get(0)); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitFirstValue(this, context); + } + + @Override + public DataType getDataType() { + return child().getDataType(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java new file mode 100644 index 00000000000000..2a3f97408d3a3a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lag.java @@ -0,0 +1,84 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ImplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.IntegerType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** Window function: lag */ +public class Lag extends WindowFunction implements TernaryExpression, PropagateNullable, ImplicitlyCastableSignature { + + public Lag(Expression child) { + this(child, Literal.of(1), Literal.of(null)); + } + + public Lag(Expression child, Expression offset) { + this(child, offset, Literal.of(null)); + } + + public Lag(Expression child, Expression offset, Expression defaultValue) { + super("lag", child, offset, defaultValue); + } + + public Lag(List children) { + super("lag", children); + } + + public Expression getOffset() { + return child(1); + } + + public Expression getDefaultValue() { + return child(2); + } + + @Override + public Lag withChildren(List children) { + Preconditions.checkArgument(children.size() >= 1 && children.size() <= 3); + return new Lag(children); + } + + @Override + public List getSignatures() { + return ImmutableList.of(FunctionSignature.ret(getArgument(0).getDataType()) + .args(getArgument(0).getDataType(), IntegerType.INSTANCE, getArgument(0).getDataType())); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitLag(this, context); + } + + @Override + public DataType getDataType() { + return child(0).getDataType(); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/LastValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/LastValue.java new file mode 100644 index 00000000000000..287a91d026019d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/LastValue.java @@ -0,0 +1,42 @@ +// 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.window; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; + +/** + * Window function: Last_value() + */ +public class LastValue extends FirstOrLastValue { + + public LastValue(Expression child) { + super("last_value", child); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitLastValue(this, context); + } + + @Override + public DataType getDataType() { + return child(0).getDataType(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java new file mode 100644 index 00000000000000..2028e21b927bb0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Lead.java @@ -0,0 +1,85 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.shape.TernaryExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.IntegerType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Window function: Lead() + */ +public class Lead extends WindowFunction implements TernaryExpression, PropagateNullable, ExplicitlyCastableSignature { + + public Lead(Expression child) { + this(child, Literal.of(1), Literal.of(null)); + } + + public Lead(Expression child, Expression offset) { + this(child, offset, Literal.of(null)); + } + + public Lead(Expression child, Expression offset, Expression defaultValue) { + super("lead", child, offset, defaultValue); + } + + public Lead(List children) { + super("lead", children); + } + + public Expression getOffset() { + return child(1); + } + + public Expression getDefaultValue() { + return child(2); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitLead(this, context); + } + + @Override + public List getSignatures() { + return ImmutableList.of(FunctionSignature.ret(getArgument(0).getDataType()) + .args(getArgument(0).getDataType(), IntegerType.INSTANCE, getArgument(0).getDataType())); + } + + @Override + public Lead withChildren(List children) { + Preconditions.checkArgument(children.size() >= 1 && children.size() <= 3); + return new Lead(children); + } + + @Override + public DataType getDataType() { + return child(0).getDataType(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Ntile.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Ntile.java new file mode 100644 index 00000000000000..16321d6828004b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Ntile.java @@ -0,0 +1,76 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.IntegerType; +import org.apache.doris.nereids.types.LargeIntType; +import org.apache.doris.nereids.types.SmallIntType; +import org.apache.doris.nereids.types.TinyIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * window function: Ntile() + */ +public class Ntile extends WindowFunction implements LeafExpression, AlwaysNotNullable, ExplicitlyCastableSignature { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(BigIntType.INSTANCE).args(TinyIntType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(SmallIntType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(IntegerType.INSTANCE), + FunctionSignature.ret(BigIntType.INSTANCE).args(BigIntType.INSTANCE), + FunctionSignature.ret(LargeIntType.INSTANCE).args(LargeIntType.INSTANCE) + ); + + private Expression buckets; + + public Ntile(Expression buckets) { + super("ntile", buckets); + this.buckets = buckets; + } + + public Expression getBuckets() { + return buckets; + } + + @Override + public Ntile withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new Ntile(children.get(0)); + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitNtile(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Rank.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Rank.java new file mode 100644 index 00000000000000..231530a9ecb2e3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/Rank.java @@ -0,0 +1,61 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Window function: Rank() + */ +public class Rank extends WindowFunction implements AlwaysNotNullable, LeafExpression { + + public Rank() { + super("rank"); + } + + @Override + public List getSignatures() { + return ImmutableList.of(FunctionSignature.ret(BigIntType.INSTANCE).args()); + } + + @Override + public FunctionSignature searchSignature(List signatures) { + return signatures.get(0); + } + + @Override + public Rank withChildren(List children) { + Preconditions.checkArgument(children.size() == 0); + return new Rank(); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitRank(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RowNumber.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RowNumber.java new file mode 100644 index 00000000000000..1755e2258a3ad9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/RowNumber.java @@ -0,0 +1,62 @@ +// 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.window; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.BigIntType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Window function: Row_number() + */ +public class RowNumber extends WindowFunction implements AlwaysNotNullable, LeafExpression { + + public RowNumber() { + super("row_number"); + } + + @Override + public List getSignatures() { + return ImmutableList.of(FunctionSignature.ret(BigIntType.INSTANCE).args()); + } + + @Override + public FunctionSignature searchSignature(List signatures) { + return signatures.get(0); + } + + @Override + public RowNumber withChildren(List children) { + Preconditions.checkArgument(children.size() == 0); + return new RowNumber(); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitRowNumber(this, context); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java new file mode 100644 index 00000000000000..2f825c17bdeec4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java @@ -0,0 +1,56 @@ +// 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.window; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; + +import java.util.List; +import java.util.Objects; + +/** + * Window functions, as known as analytic functions. + */ +public abstract class WindowFunction extends BoundFunction { + + public WindowFunction(String name, Expression... arguments) { + super(name, arguments); + } + + public WindowFunction(String name, List children) { + super(name, children); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WindowFunction that = (WindowFunction) o; + return Objects.equals(getName(), that.getName()) + && Objects.equals(children, that.children); + } + + @Override + public int hashCode() { + return Objects.hash(getName(), children); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index cd5c6152856a68..c8aff7e4d2917e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -73,12 +73,15 @@ import org.apache.doris.nereids.trees.expressions.VariableDesc; import org.apache.doris.nereids.trees.expressions.VirtualSlotReference; import org.apache.doris.nereids.trees.expressions.WhenClause; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; 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.scalar.GroupingScalarFunction; import org.apache.doris.nereids.trees.expressions.functions.scalar.ScalarFunction; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; +import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; 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.BooleanLiteral; @@ -103,7 +106,8 @@ */ public abstract class ExpressionVisitor implements ScalarFunctionVisitor, AggregateFunctionVisitor, - TableValuedFunctionVisitor, TableGeneratingFunctionVisitor { + TableValuedFunctionVisitor, TableGeneratingFunctionVisitor, + WindowFunctionVisitor { public abstract R visit(Expression expr, C context); @@ -127,6 +131,11 @@ public R visitTableGeneratingFunction(TableGeneratingFunction tableGeneratingFun return visitBoundFunction(tableGeneratingFunction, context); } + @Override + public R visitWindowFunction(WindowFunction windowFunction, C context) { + return visitBoundFunction(windowFunction, context); + } + public R visitBoundFunction(BoundFunction boundFunction, C context) { return visit(boundFunction, context); } @@ -411,6 +420,14 @@ public R visitOrderExpression(OrderExpression orderExpression, C context) { return visit(orderExpression, context); } + public R visitWindow(WindowExpression windowExpression, C context) { + return visit(windowExpression, context); + } + + public R visitWindowFrame(WindowFrame windowFrame, C context) { + return visit(windowFrame, context); + } + /* ******************************************************************************************** * Unbound expressions * ********************************************************************************************/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/WindowFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/WindowFunctionVisitor.java new file mode 100644 index 00000000000000..3a97c744be6f0a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/WindowFunctionVisitor.java @@ -0,0 +1,66 @@ +// 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.visitor; + +import org.apache.doris.nereids.trees.expressions.functions.window.DenseRank; +import org.apache.doris.nereids.trees.expressions.functions.window.FirstValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lag; +import org.apache.doris.nereids.trees.expressions.functions.window.LastValue; +import org.apache.doris.nereids.trees.expressions.functions.window.Lead; +import org.apache.doris.nereids.trees.expressions.functions.window.Ntile; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber; +import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; + +/** WindowFunctionVisitor. */ +public interface WindowFunctionVisitor { + + R visitWindowFunction(WindowFunction windowFunction, C context); + + default R visitDenseRank(DenseRank denseRank, C context) { + return visitWindowFunction(denseRank, context); + } + + default R visitFirstValue(FirstValue firstValue, C context) { + return visitWindowFunction(firstValue, context); + } + + default R visitLag(Lag lag, C context) { + return visitWindowFunction(lag, context); + } + + default R visitLastValue(LastValue lastValue, C context) { + return visitWindowFunction(lastValue, context); + } + + default R visitLead(Lead lead, C context) { + return visitWindowFunction(lead, context); + } + + default R visitNtile(Ntile ntile, C context) { + return visitWindowFunction(ntile, context); + } + + default R visitRank(Rank rank, C context) { + return visitWindowFunction(rank, context); + } + + default R visitRowNumber(RowNumber rowNumber, C context) { + return visitWindowFunction(rowNumber, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index ab355837fe2045..c8e1a2222e5252 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -25,6 +25,7 @@ public enum PlanType { // logical plan LOGICAL_CTE, + LOGICAL_WINDOW, LOGICAL_SUBQUERY_ALIAS, LOGICAL_UNBOUND_ONE_ROW_RELATION, LOGICAL_EMPTY_RELATION, @@ -57,6 +58,7 @@ public enum PlanType { GROUP_PLAN, // physical plan + PHYSICAL_WINDOW, PHYSICAL_EMPTY_RELATION, PHYSICAL_ONE_ROW_RELATION, PHYSICAL_OLAP_SCAN, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Window.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Window.java new file mode 100644 index 00000000000000..35f6547b8af7c0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Window.java @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.algebra; + +import org.apache.doris.analysis.AnalyticWindow; +import org.apache.doris.analysis.Expr; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.glue.translator.ExpressionTranslator; +import org.apache.doris.nereids.glue.translator.PlanTranslatorContext; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundType; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundary; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameUnitsType; +import org.apache.doris.nereids.trees.expressions.literal.Literal; + +import java.math.BigDecimal; +import java.util.List; + +/** + * interface for LogicalWindow and PhysicalWindow + */ +public interface Window { + + List getWindowExpressions(); + + /** + * translate WindowFrame to AnalyticWindow + */ + default AnalyticWindow translateWindowFrame(WindowFrame windowFrame, PlanTranslatorContext context) { + FrameUnitsType frameUnits = windowFrame.getFrameUnits(); + FrameBoundary leftBoundary = windowFrame.getLeftBoundary(); + FrameBoundary rightBoundary = windowFrame.getRightBoundary(); + + AnalyticWindow.Type type = frameUnits == FrameUnitsType.ROWS + ? AnalyticWindow.Type.ROWS : AnalyticWindow.Type.RANGE; + + AnalyticWindow.Boundary left = withFrameBoundary(leftBoundary, context); + AnalyticWindow.Boundary right = withFrameBoundary(rightBoundary, context); + + return new AnalyticWindow(type, left, right); + } + + /** + * translate FrameBoundary to Boundary + */ + default AnalyticWindow.Boundary withFrameBoundary(FrameBoundary boundary, PlanTranslatorContext context) { + FrameBoundType boundType = boundary.getFrameBoundType(); + BigDecimal offsetValue = null; + Expr e = null; + if (boundary.hasOffset()) { + Expression boundOffset = boundary.getBoundOffset().get(); + offsetValue = new BigDecimal(((Literal) boundOffset).getDouble()); + e = ExpressionTranslator.translate(boundOffset, context); + } + + switch (boundType) { + case UNBOUNDED_PRECEDING: + return new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.UNBOUNDED_PRECEDING, null); + case UNBOUNDED_FOLLOWING: + return new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.UNBOUNDED_FOLLOWING, null); + case CURRENT_ROW: + return new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.CURRENT_ROW, null); + case PRECEDING: + return new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.PRECEDING, e, offsetValue); + case FOLLOWING: + return new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.FOLLOWING, e, offsetValue); + default: + throw new AnalysisException("This WindowFrame hasn't be resolved in REWRITE"); + } + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java new file mode 100644 index 00000000000000..c29f9aba7a5abc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java @@ -0,0 +1,156 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Window; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * logical node to deal with window functions; + */ +public class LogicalWindow extends LogicalUnary implements Window { + + // List> + private final List windowExpressions; + + private final boolean isChecked; + + public LogicalWindow(List windowExpressions, CHILD_TYPE child) { + this(windowExpressions, false, Optional.empty(), Optional.empty(), child); + } + + public LogicalWindow(List windowExpressions, boolean isChecked, CHILD_TYPE child) { + this(windowExpressions, isChecked, Optional.empty(), Optional.empty(), child); + } + + public LogicalWindow(List windowExpressions, boolean isChecked, + Optional groupExpression, Optional logicalProperties, + CHILD_TYPE child) { + super(PlanType.LOGICAL_WINDOW, groupExpression, logicalProperties, child); + this.windowExpressions = ImmutableList.copyOf(Objects.requireNonNull(windowExpressions, "output expressions" + + "in LogicalWindow cannot be null")); + this.isChecked = isChecked; + } + + public boolean isChecked() { + return isChecked; + } + + @Override + public List getWindowExpressions() { + return windowExpressions; + } + + @Override + public List getExpressions() { + return windowExpressions; + } + + public LogicalWindow withChecked(List windowExpressions, Plan child) { + return new LogicalWindow(windowExpressions, true, Optional.empty(), Optional.empty(), child); + } + + @Override + public LogicalUnary withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new LogicalWindow<>(windowExpressions, isChecked, children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalWindow(this, context); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalWindow<>(windowExpressions, isChecked, + groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withLogicalProperties(Optional logicalProperties) { + return new LogicalWindow<>(windowExpressions, isChecked, + Optional.empty(), logicalProperties, child()); + } + + /** + * LogicalWindow need to add child().getOutput() as its outputs, to resolve patterns like the following + * after the implementation rule LogicalWindowToPhysicalWindow: + * + * origin: + * LogicalProject( projects = [row_number as `row_number`, rank as `rank`] + * +--LogicalWindow( windowExpressions = [row_number() over(order by c1), rank() over(order by c2)] + * + * after(not show PhysicalLogicalQuickSort generated by enforcer): + * PhysicalProject( projects = [row_number as `row_number`, rank as `rank`] + * +--PhysicalWindow( windowExpressions = [row_number() over(order by c1)]) + * +--PhysicalWindow( windowExpressions = [rank() over(order by c2)]) + * + * if we don't add child().getOutput(), the top-PhysicalProject cannot find rank() + */ + @Override + public List computeOutput() { + return new ImmutableList.Builder() + .addAll(child().getOutput()) + .addAll(windowExpressions.stream() + .map(NamedExpression::toSlot) + .collect(ImmutableList.toImmutableList())) + .build(); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalWindow", + "windowExpressions", windowExpressions, + "isChecked", isChecked + ); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalWindow that = (LogicalWindow) o; + return Objects.equals(windowExpressions, that.windowExpressions) + && isChecked == that.isChecked; + } + + @Override + public int hashCode() { + return Objects.hash(windowExpressions, isChecked); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalQuickSort.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalQuickSort.java index 32bf4c2d0a3f97..b032a68df415c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalQuickSort.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalQuickSort.java @@ -46,18 +46,17 @@ public PhysicalQuickSort(List orderKeys, /** * Constructor of PhysicalHashJoinNode. */ - public PhysicalQuickSort(List orderKeys, - Optional groupExpression, LogicalProperties logicalProperties, - CHILD_TYPE child) { + public PhysicalQuickSort(List orderKeys, Optional groupExpression, + LogicalProperties logicalProperties, CHILD_TYPE child) { super(PlanType.PHYSICAL_QUICK_SORT, orderKeys, groupExpression, logicalProperties, child); } /** * Constructor of PhysicalHashJoinNode. */ - public PhysicalQuickSort(List orderKeys, - Optional groupExpression, LogicalProperties logicalProperties, - PhysicalProperties physicalProperties, StatsDeriveResult statsDeriveResult, CHILD_TYPE child) { + public PhysicalQuickSort(List orderKeys, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, + StatsDeriveResult statsDeriveResult, CHILD_TYPE child) { super(PlanType.PHYSICAL_QUICK_SORT, orderKeys, groupExpression, logicalProperties, physicalProperties, statsDeriveResult, child); } @@ -86,14 +85,14 @@ public PhysicalQuickSort withLogicalProperties(Optional withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, StatsDeriveResult statsDeriveResult) { - return new PhysicalQuickSort<>(orderKeys, Optional.empty(), getLogicalProperties(), physicalProperties, - statsDeriveResult, child()); + return new PhysicalQuickSort<>(orderKeys, Optional.empty(), getLogicalProperties(), + physicalProperties, statsDeriveResult, child()); } @Override public String toString() { return Utils.toSqlString("PhysicalQuickSort", - "orderKeys", orderKeys + "orderKeys", orderKeys ); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java new file mode 100644 index 00000000000000..276f47bbb2ec41 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWindow.java @@ -0,0 +1,164 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.physical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.properties.RequireProperties; +import org.apache.doris.nereids.properties.RequirePropertiesSupplier; +import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Window; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.StatsDeriveResult; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * physical node for window function + */ +public class PhysicalWindow extends PhysicalUnary implements Window, + RequirePropertiesSupplier> { + + private final WindowFrameGroup windowFrameGroup; + private final RequireProperties requireProperties; + + public PhysicalWindow(WindowFrameGroup windowFrameGroup, RequireProperties requireProperties, + LogicalProperties logicalProperties, CHILD_TYPE child) { + this(windowFrameGroup, requireProperties, Optional.empty(), logicalProperties, child); + } + + /** constructor for PhysicalWindow */ + public PhysicalWindow(WindowFrameGroup windowFrameGroup, RequireProperties requireProperties, + Optional groupExpression, LogicalProperties logicalProperties, + CHILD_TYPE child) { + super(PlanType.PHYSICAL_WINDOW, groupExpression, logicalProperties, child); + this.windowFrameGroup = Objects.requireNonNull(windowFrameGroup, "windowFrameGroup in PhysicalWindow" + + "cannot be null"); + this.requireProperties = requireProperties; + } + + /** constructor for PhysicalWindow */ + public PhysicalWindow(WindowFrameGroup windowFrameGroup, RequireProperties requireProperties, + Optional groupExpression, LogicalProperties logicalProperties, + PhysicalProperties physicalProperties, StatsDeriveResult statsDeriveResult, + CHILD_TYPE child) { + super(PlanType.PHYSICAL_WINDOW, groupExpression, logicalProperties, physicalProperties, + statsDeriveResult, child); + this.windowFrameGroup = Objects.requireNonNull(windowFrameGroup, "windowFrameGroup in PhysicalWindow" + + "cannot be null"); + this.requireProperties = requireProperties; + } + + @Override + public List getWindowExpressions() { + return windowFrameGroup.getGroups(); + } + + public WindowFrameGroup getWindowFrameGroup() { + return windowFrameGroup; + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalWindow(this, context); + } + + @Override + public List getExpressions() { + return windowFrameGroup.getGroups(); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalWindow", + "windowFrameGroup", windowFrameGroup, + "requiredProperties", requireProperties + ); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PhysicalWindow that = (PhysicalWindow) o; + return Objects.equals(windowFrameGroup, that.windowFrameGroup) + && Objects.equals(requireProperties, that.requireProperties); + } + + @Override + public int hashCode() { + return Objects.hash(windowFrameGroup, requireProperties); + } + + @Override + public RequireProperties getRequireProperties() { + return requireProperties; + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkState(children.size() == 1); + return new PhysicalWindow<>(windowFrameGroup, requireProperties, Optional.empty(), + getLogicalProperties(), children.get(0)); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new PhysicalWindow<>(windowFrameGroup, requireProperties, groupExpression, + getLogicalProperties(), child()); + } + + @Override + public Plan withLogicalProperties(Optional logicalProperties) { + return new PhysicalWindow<>(windowFrameGroup, requireProperties, Optional.empty(), + logicalProperties.get(), child()); + } + + @Override + public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, + StatsDeriveResult statsDeriveResult) { + return new PhysicalWindow<>(windowFrameGroup, requireProperties, Optional.empty(), + getLogicalProperties(), physicalProperties, statsDeriveResult, child()); + } + + @Override + public PhysicalWindow withRequireAndChildren(RequireProperties requireProperties, List children) { + Preconditions.checkArgument(children.size() == 1); + return withRequirePropertiesAndChild(requireProperties, children.get(0)); + } + + public PhysicalWindow withRequirePropertiesAndChild(RequireProperties requireProperties, + C newChild) { + return new PhysicalWindow<>(windowFrameGroup, requireProperties, Optional.empty(), + getLogicalProperties(), physicalProperties, statsDeriveResult, newChild); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java index af464b0fb1f7e4..0723b6444ec417 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java @@ -52,6 +52,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; @@ -79,6 +80,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; /** * Base class for the processing of logical and physical plan. @@ -235,6 +237,10 @@ public R visitLogicalGenerate(LogicalGenerate generate, C contex return visit(generate, context); } + public R visitLogicalWindow(LogicalWindow window, C context) { + return visit(window, context); + } + // ******************************* // Physical plans // ******************************* @@ -287,6 +293,10 @@ public R visitPhysicalQuickSort(PhysicalQuickSort sort, C contex return visitAbstractPhysicalSort(sort, context); } + public R visitPhysicalWindow(PhysicalWindow window, C context) { + return visit(window, context); + } + public R visitPhysicalTopN(PhysicalTopN topN, C context) { return visit(topN, context); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index 17ea5e5e375981..043bad1d33bb82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -497,6 +497,10 @@ public boolean isNullType() { return this instanceof NullType; } + public boolean isIntegralType() { + return this instanceof IntegralType; + } + public boolean isNumericType() { return this instanceof NumericType; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticEvalNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticEvalNode.java index bbe8ef80d0faaf..fb5a63b10f2736 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticEvalNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticEvalNode.java @@ -99,6 +99,32 @@ public AnalyticEvalNode( nullableTupleIds = Sets.newHashSet(input.getNullableTupleIds()); } + // constructor used in Nereids + public AnalyticEvalNode( + PlanNodeId id, PlanNode input, List analyticFnCalls, + List partitionExprs, List orderByElements, + AnalyticWindow analyticWindow, TupleDescriptor intermediateTupleDesc, + TupleDescriptor outputTupleDesc, Expr partitionByEq, Expr orderByEq, + TupleDescriptor bufferedTupleDesc) { + super(id, input.getTupleIds(), "ANALYTIC", StatisticalType.ANALYTIC_EVAL_NODE); + Preconditions.checkState(!tupleIds.contains(outputTupleDesc.getId())); + // we're materializing the input row augmented with the analytic output tuple + tupleIds.add(outputTupleDesc.getId()); + this.analyticFnCalls = analyticFnCalls; + this.partitionExprs = partitionExprs; + this.substitutedPartitionExprs = partitionExprs; + this.orderByElements = orderByElements; + this.analyticWindow = analyticWindow; + this.intermediateTupleDesc = intermediateTupleDesc; + this.outputTupleDesc = outputTupleDesc; + this.logicalToPhysicalSmap = new ExprSubstitutionMap(); + this.partitionByEq = partitionByEq; + this.orderByEq = orderByEq; + this.bufferedTupleDesc = bufferedTupleDesc; + children.add(input); + nullableTupleIds = Sets.newHashSet(input.getNullableTupleIds()); + } + public List getPartitionExprs() { return partitionExprs; } @@ -218,9 +244,7 @@ public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { List strings = Lists.newArrayList(); for (Expr fnCall : analyticFnCalls) { - strings.add("["); - strings.add(fnCall.toSql()); - strings.add("]"); + strings.add("[" + fnCall.toSql() + "]"); } output.append(Joiner.on(", ").join(strings)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticPlanner.java index e7e7533ff60a4a..46a21c6ed4bc1d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/AnalyticPlanner.java @@ -92,10 +92,10 @@ public AnalyticPlanner(AnalyticInfo analyticInfo, Analyzer analyzer, PlannerCont * a subset of the grouping exprs which should be used for the aggregate * hash partitioning during the parallelization of 'root'. * TODO: when generating sort orders for the sort groups, optimize the ordering - * of the partition exprs (so that subsequent sort operations see the input sorted - * on a prefix of their required sort exprs) + * of the partition exprs (so that subsequent sort operations see the input sorted + * on a prefix of their required sort exprs) * TODO: when merging sort groups, recognize equivalent exprs - * (using the equivalence classes) rather than looking for expr equality + * (using the equivalence classes) rather than looking for expr equality * @throws AnalysisException * @throws UserException */ 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 cf120e210f1857..f32fbb2e53a453 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.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -85,7 +86,6 @@ public void testPostProcessor() { @Test public void testParseCTE() { - // Just for debug; will be completed before merged; NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan; String cteSql1 = "with t1 as (select s_suppkey from supplier where s_suppkey < 10) select * from t1"; @@ -106,6 +106,25 @@ public void testParseCTE() { Assertions.assertEquals(columnAliases.get().size(), 2); } + @Test + public void testParseWindowFunctions() { + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan logicalPlan; + String windowSql1 = "select k1, rank() over(partition by k1 order by k1) as ranking from t1"; + logicalPlan = nereidsParser.parseSingle(windowSql1); + Assertions.assertEquals(PlanType.LOGICAL_PROJECT, logicalPlan.getType()); + Assertions.assertEquals(((LogicalProject) logicalPlan).getProjects().size(), 2); + + String windowSql2 = "select k1, sum(k2), rank() over(partition by k1 order by k1) as ranking from t1 group by k1"; + logicalPlan = nereidsParser.parseSingle(windowSql2); + Assertions.assertEquals(PlanType.LOGICAL_AGGREGATE, logicalPlan.getType()); + Assertions.assertEquals(((LogicalAggregate) logicalPlan).getOutputExpressions().size(), 3); + + String windowSql3 = "select rank() over from t1"; + parsePlan(windowSql3).assertThrowsExactly(ParseException.class) + .assertMessageContains("mismatched input 'from' expecting '('"); + } + @Test public void testExplainNormal() { String sql = "explain select `AD``D` from t1 where a = 1"; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionTest.java new file mode 100644 index 00000000000000..9faa34487c0c8e --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/CheckAndStandardizeWindowFunctionTest.java @@ -0,0 +1,260 @@ +// 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.logical; + +import org.apache.doris.nereids.properties.OrderKey; +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.OrderExpression; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundary; +import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameUnitsType; +import org.apache.doris.nereids.trees.expressions.functions.window.DenseRank; +import org.apache.doris.nereids.trees.expressions.functions.window.Lag; +import org.apache.doris.nereids.trees.expressions.functions.window.Lead; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber; +import org.apache.doris.nereids.trees.expressions.functions.window.WindowFunction; +import org.apache.doris.nereids.trees.expressions.literal.DoubleLiteral; +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.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.logical.RelationUtil; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.nereids.util.PlanConstructor; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.util.List; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class CheckAndStandardizeWindowFunctionTest implements PatternMatchSupported { + + private LogicalPlan rStudent; + private NamedExpression gender; + private NamedExpression age; + private List partitionKeyList; + private List orderKeyList; + private WindowFrame defaultWindowFrame; + + @BeforeAll + public final void beforeAll() { + rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, ImmutableList.of()); + + gender = rStudent.getOutput().get(1).toSlot(); + age = rStudent.getOutput().get(3).toSlot(); + partitionKeyList = ImmutableList.of(gender); + orderKeyList = ImmutableList.of(new OrderExpression(new OrderKey(age, true, true))); + defaultWindowFrame = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + } + + @Test + public void testRankAndDenseRank() { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + List funcList = Lists.newArrayList(new Rank(), new DenseRank()); + + for (WindowFunction func : funcList) { + WindowExpression window = new WindowExpression(func, partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + List outputExpressions = Lists.newArrayList(windowAlias); + Plan root = new LogicalProject<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .applyTopDown(new CheckAndStandardizeWindowFunctionAndFrame()) + .matches( + logicalWindow() + .when(logicalWindow -> { + WindowExpression newWindow = (WindowExpression) logicalWindow.getWindowExpressions().get(0).child(0); + return newWindow.getWindowFrame().get().equals(requiredFrame); + }) + ); + } + } + + @Test + public void testRowNumber() { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newCurrentRowBoundary()); + WindowExpression window = new WindowExpression(new RowNumber(), partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + List outputExpressions = Lists.newArrayList(windowAlias); + Plan root = new LogicalWindow<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .applyTopDown(new CheckAndStandardizeWindowFunctionAndFrame()) + .matches( + logicalWindow() + .when(logicalWindow -> { + WindowExpression newWindow = (WindowExpression) logicalWindow.getWindowExpressions().get(0).child(0); + return newWindow.getWindowFrame().get().equals(requiredFrame); + }) + ); + } + + @Test + public void testLead() { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newFollowingBoundary(new IntegerLiteral(5))); + WindowExpression window = new WindowExpression(new Lead(age, new IntegerLiteral(5), new IntegerLiteral(0)), + partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + List outputExpressions = Lists.newArrayList(windowAlias); + Plan root = new LogicalWindow<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .applyTopDown(new CheckAndStandardizeWindowFunctionAndFrame()) + .matches( + logicalWindow() + .when(logicalWindow -> { + WindowExpression newWindow = (WindowExpression) logicalWindow.getWindowExpressions().get(0).child(0); + return newWindow.getWindowFrame().get().equals(requiredFrame); + }) + ); + } + + @Test + public void testLag() { + WindowFrame requiredFrame = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(), FrameBoundary.newPrecedingBoundary(new IntegerLiteral(5))); + WindowExpression window = new WindowExpression(new Lag(age, new IntegerLiteral(5), new IntegerLiteral(0)), + partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + List outputExpressions = Lists.newArrayList(windowAlias); + Plan root = new LogicalWindow<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .applyTopDown(new CheckAndStandardizeWindowFunctionAndFrame()) + .matches( + logicalWindow() + .when(logicalWindow -> { + WindowExpression newWindow = (WindowExpression) logicalWindow.getWindowExpressions().get(0).child(0); + return newWindow.getWindowFrame().get().equals(requiredFrame); + }) + ); + } + + @Test + public void testCheckWindowFrameBeforeFunc0() { + WindowExpression window = new WindowExpression(new Rank(), partitionKeyList, Lists.newArrayList(), defaultWindowFrame); + String errorMsg = "WindowFrame clause requires OrderBy clause"; + + forCheckWindowFrameBeforeFunc(window, errorMsg); + } + + @Test + public void testCheckWindowFrameBeforeFunc1() { + WindowFrame windowFrame1 = new WindowFrame(FrameUnitsType.ROWS, FrameBoundary.newFollowingBoundary()); + String errorMsg1 = "WindowFrame in any window function cannot use UNBOUNDED FOLLOWING as left boundary"; + forCheckWindowFrameBeforeFunc(windowFrame1, errorMsg1); + + WindowFrame windowFrame2 = new WindowFrame(FrameUnitsType.ROWS, + WindowFrame.FrameBoundary.newFollowingBoundary(new IntegerLiteral(3)), FrameBoundary.newCurrentRowBoundary()); + String errorMsg2 = "WindowFrame with FOLLOWING left boundary requires UNBOUNDED FOLLOWING or FOLLOWING right boundary"; + forCheckWindowFrameBeforeFunc(windowFrame2, errorMsg2); + } + + @Test + public void testCheckWindowFrameBeforeFunc2() { + WindowFrame windowFrame1 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newCurrentRowBoundary(), FrameBoundary.newPrecedingBoundary()); + String errorMsg1 = "WindowFrame in any window function cannot use UNBOUNDED PRECEDING as right boundary"; + forCheckWindowFrameBeforeFunc(windowFrame1, errorMsg1); + + WindowFrame windowFrame2 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newCurrentRowBoundary(), FrameBoundary.newPrecedingBoundary(new IntegerLiteral(3))); + String errorMsg2 = "WindowFrame with PRECEDING right boundary requires UNBOUNDED PRECEDING or PRECEDING left boundary"; + forCheckWindowFrameBeforeFunc(windowFrame2, errorMsg2); + } + + @Test + public void testCheckWindowFrameBeforeFunc3() { + WindowFrame windowFrame1 = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newPrecedingBoundary(new IntegerLiteral(3)), FrameBoundary.newCurrentRowBoundary()); + WindowFrame windowFrame2 = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newCurrentRowBoundary(), FrameBoundary.newFollowingBoundary(new IntegerLiteral(3))); + WindowFrame windowFrame3 = new WindowFrame(FrameUnitsType.RANGE, + FrameBoundary.newCurrentRowBoundary(), FrameBoundary.newCurrentRowBoundary()); + String errorMsg = "WindowFrame with RANGE must use both UNBOUNDED boundary or one UNBOUNDED boundary and one CURRENT ROW"; + + forCheckWindowFrameBeforeFunc(windowFrame1, errorMsg); + forCheckWindowFrameBeforeFunc(windowFrame2, errorMsg); + forCheckWindowFrameBeforeFunc(windowFrame3, errorMsg); + } + + @Test + public void testCheckWindowFrameBeforeFunc4() { + WindowFrame windowFrame1 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(new IntegerLiteral(-3)), FrameBoundary.newCurrentRowBoundary()); + String errorMsg1 = "BoundOffset of WindowFrame must be positive"; + forCheckWindowFrameBeforeFunc(windowFrame1, errorMsg1); + + WindowFrame windowFrame2 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(new DoubleLiteral(3.5)), FrameBoundary.newCurrentRowBoundary()); + String errorMsg2 = "BoundOffset of ROWS WindowFrame must be an Integer"; + forCheckWindowFrameBeforeFunc(windowFrame2, errorMsg2); + } + + @Test + public void testCheckWindowFrameBeforeFunc5() { + WindowFrame windowFrame1 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newPrecedingBoundary(new IntegerLiteral(3)), FrameBoundary.newPrecedingBoundary(new IntegerLiteral(4))); + String errorMsg1 = "WindowFrame with PRECEDING boundary requires that leftBoundOffset >= rightBoundOffset"; + forCheckWindowFrameBeforeFunc(windowFrame1, errorMsg1); + + WindowFrame windowFrame2 = new WindowFrame(FrameUnitsType.ROWS, + FrameBoundary.newFollowingBoundary(new IntegerLiteral(5)), FrameBoundary.newFollowingBoundary(new IntegerLiteral(4))); + String errorMsg2 = "WindowFrame with FOLLOWING boundary requires that leftBoundOffset >= rightBoundOffset"; + forCheckWindowFrameBeforeFunc(windowFrame2, errorMsg2); + } + + private void forCheckWindowFrameBeforeFunc(WindowFrame windowFrame, String errorMsg) { + WindowExpression window = new WindowExpression(new Rank(), partitionKeyList, orderKeyList, windowFrame); + forCheckWindowFrameBeforeFunc(window, errorMsg); + } + + private void forCheckWindowFrameBeforeFunc(WindowExpression window, String errorMsg) { + Alias windowAlias = new Alias(window, window.toSql()); + List outputExpressions = Lists.newArrayList(windowAlias); + Plan root = new LogicalWindow<>(outputExpressions, rStudent); + + Exception exception = Assertions.assertThrows(Exception.class, () -> { + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .applyTopDown(new CheckAndStandardizeWindowFunctionAndFrame()); + }, "Not throw expected exception."); + Assertions.assertTrue(exception.getMessage().contains(errorMsg)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpressionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpressionTest.java new file mode 100644 index 00000000000000..39d62974a9aca0 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/ExtractAndNormalizeWindowExpressionTest.java @@ -0,0 +1,233 @@ +// 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.logical; + +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.trees.expressions.Add; +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.OrderExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +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; +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.trees.plans.logical.RelationUtil; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.nereids.util.PlanConstructor; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.util.List; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class ExtractAndNormalizeWindowExpressionTest implements PatternMatchSupported { + + private LogicalPlan rStudent; + + @BeforeAll + public final void beforeAll() { + rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, ImmutableList.of()); + } + + @Test + public void testSimpleWindowFunction() { + NamedExpression id = rStudent.getOutput().get(0).toSlot(); + NamedExpression gender = rStudent.getOutput().get(1).toSlot(); + NamedExpression age = rStudent.getOutput().get(3).toSlot(); + + List partitionKeyList = ImmutableList.of(gender); + List orderKeyList = ImmutableList.of(new OrderExpression( + new OrderKey(age, true, true))); + WindowExpression window = new WindowExpression(new Rank(), partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + + List outputExpressions = Lists.newArrayList(id, windowAlias); + Plan root = new LogicalProject<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .matchesFromRoot( + logicalProject( + logicalWindow( + logicalProject( + logicalOlapScan() + ).when(project -> { + List projects = project.getProjects(); + return projects.get(0).equals(id) + && projects.get(1).equals(gender) + && projects.get(2).equals(age); + }) + ).when(logicalWindow -> logicalWindow.getWindowExpressions().get(0).equals(windowAlias)) + ).when(project -> { + List projects = project.getProjects(); + return projects.get(0).equals(id) + && projects.get(1) instanceof Alias; + }) + ); + } + + /*- + * original plan: + * LogicalProject (output: [id#0, rank() over(partition by id#0+2 order by age#3+2) AS `Window.toSql()`#4) + * +--ScanOlapTable (student.student, output: [id#0, gender#1, name#2, age#3]) + * + * after rewrite: + * LogicalProject ( projects=[id#0, `Window.toSql()`#4] ) + * +--LogicalWindow (windowExpressions: [rank() over(partition by (id + 2)#5 order by (age3 + 2)#6 AS `Window.toSql()`#4) + * +--LogicalProject( projects=[id#0, (id#0 + 2) AS `(id + 2)`#5, (age#3 + 2) AS `(age + 2)`#6] ) + * +--ScanOlapTable (student.student, output: [id#0, gender#1, name#2, age#3]) + */ + @Test + public void testComplexWindowFunction() { + NamedExpression id = rStudent.getOutput().get(0).toSlot(); + NamedExpression age = rStudent.getOutput().get(3).toSlot(); + + List partitionKeyList = ImmutableList.of(new Add(id, new IntegerLiteral(2))); + List orderKeyList = ImmutableList.of(new OrderExpression( + new OrderKey(new Add(age, new IntegerLiteral(2)), true, true))); + WindowExpression window = new WindowExpression(new Rank(), partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + + List outputExpressions = Lists.newArrayList(id, windowAlias); + Plan root = new LogicalProject<>(outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .matchesFromRoot( + logicalProject( + logicalWindow( + logicalProject( + logicalOlapScan() + ).when(project -> { + List projects = project.getProjects(); + return projects.get(1) instanceof Alias && projects.get(1).child(0).child(0).equals(id) + && projects.get(1).child(0).child(1).equals(new IntegerLiteral(2)) + && projects.get(2) instanceof Alias && projects.get(2).child(0).child(0).equals(age) + && projects.get(2).child(0).child(1).equals(new IntegerLiteral(2)) + && projects.get(0).equals(id); + }) + ).when(logicalWindow -> { + List outputs = logicalWindow.getWindowExpressions(); + WindowExpression newWindow = (WindowExpression) outputs.get(0).child(0); + Expression pk = newWindow.getPartitionKeys().get(0); + OrderExpression ok = newWindow.getOrderKeys().get(0); + return !newWindow.equals(windowAlias.child(0)) + && pk instanceof SlotReference + && ok.child() instanceof SlotReference; + }) + ).when(project -> { + List projects = project.getProjects(); + return projects.get(0) instanceof SlotReference + && projects.get(1) instanceof Alias; + }) + ); + } + + /* + * select gender, sum(id+1), sum(id+1) over(partition by gender order by id+1) + * from student + * group by gender, id + * + * original plan: + * LogicalAggregate (output: [gender#1, sum(id#0 + 1) As `sum`#4, sum(id#0 + 1) over(partition by gender#1 order by id#0 + 1) AS `Window.toSql()`#5], + * groupBy: [gender#1, id#0]) + * +--ScanOlapTable (student.student, output: [id#0, gender#1, name#2, age#3]) + * + * after rewrite: + * LogicalProject ( projects=[gender#1, sum#4, Window.toSql()#5 AS `Window.toSql()`#5] ) + * +--LogicalWindow (windowExpressions: [sum((id + 1)#7) over(partition by gender#1 order by (id + 1)#7 AS `Window.toSql()`#5) + * +--LogicalProject( projects=[gender#1, sum#4 AS `sum`#4, (id#0 + 1) AS `(id + 1)`#7] ) + * +--LogicalAggregate (output: [gender#1, id#0, sum((id + 1)#6) As `sum`#4], + * groupBy: [gender#1, id#0]) + * +--LogicalProject( projects=[gender#1, id#0, (id#0 + 1) AS `(id + 1)`#6] ) + * +--ScanOlapTable (student.student, output: [id#0, gender#1, name#2, age#3]) + */ + @Test + public void testComplexWindowFunctionTogetherWithAggregateFunction() { + NamedExpression id = rStudent.getOutput().get(0).toSlot(); + NamedExpression gender = rStudent.getOutput().get(1).toSlot(); + Add add = new Add(id, new IntegerLiteral(1)); + Alias sumAlias = new Alias(new Sum(add), "sum"); + + List partitionKeyList = ImmutableList.of(gender); + List orderKeyList = ImmutableList.of(new OrderExpression(new OrderKey(add, true, true))); + WindowExpression window = new WindowExpression(new Sum(add), partitionKeyList, orderKeyList); + Alias windowAlias = new Alias(window, window.toSql()); + + List groupKeys = Lists.newArrayList(gender, id); + List outputExpressions = Lists.newArrayList(gender, sumAlias, windowAlias); + Plan root = new LogicalAggregate<>(groupKeys, outputExpressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyTopDown(new NormalizeAggregate()) + .matches( + logicalProject( + logicalAggregate() + ).when(project -> { + // ensure that WindowExpression in LogicalAgg will not be changed + // while AggregateFunction can be normalized as usual + // when Window's function is same as AggregateFunction. + // In this example, agg function [sum(id+1)] is same as Window's function [sum(id+1) over...] + List projects = project.getProjects(); + return projects.get(1).child(0) instanceof SlotReference + && projects.get(2).equals(windowAlias); + }) + ) + .applyTopDown(new ExtractAndNormalizeWindowExpression()) + .matchesFromRoot( + logicalProject( + logicalWindow( + logicalProject( + logicalAggregate( + logicalProject( + logicalOlapScan() + ) + ) + ).when(project -> project.getProjects().get(2).child(0).equals(add)) + ).when(logicalWindow -> { + WindowExpression newWindow = (WindowExpression) logicalWindow.getWindowExpressions().get(0).child(0); + Expression func = newWindow.getFunction(); + Expression ok = newWindow.getOrderKeys().get(0).child(0); + return !newWindow.equals(windowAlias.child(0)) + && func.child(0) instanceof SlotReference + && ok instanceof SlotReference + && func.child(0).equals(ok); + }) + ).when(project -> { + List projects = project.getProjects(); + return projects.get(0).equals(gender) + && projects.get(1) instanceof SlotReference + && projects.get(2) instanceof Alias + && projects.get(2).child(0) instanceof SlotReference; + }) + ); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/LogicalWindowToPhysicalWindowTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/LogicalWindowToPhysicalWindowTest.java new file mode 100644 index 00000000000000..1abe5f56ca3742 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/LogicalWindowToPhysicalWindowTest.java @@ -0,0 +1,95 @@ +// 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.logical; + +import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; +import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; +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.WindowExpression; +import org.apache.doris.nereids.trees.expressions.WindowFrame; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.window.Rank; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.logical.RelationUtil; +import org.apache.doris.nereids.util.MemoTestUtils; +import org.apache.doris.nereids.util.PatternMatchSupported; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.nereids.util.PlanConstructor; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import org.apache.commons.collections.CollectionUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.util.List; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class LogicalWindowToPhysicalWindowTest implements PatternMatchSupported { + + private Plan rStudent; + + @BeforeAll + public final void beforeAll() { + rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + ImmutableList.of("")); + } + + /** + * select rank() over(partition by id, gender), sum(age) over(partition by gender, id, gender) from student + * + * rank and sum have same WindowFrame and compatible partitionKeys, so they can be put to one PhysicalWindow + */ + @Test + public void testPartitionKeyCompatible() { + NamedExpression id = rStudent.getOutput().get(0).toSlot(); + NamedExpression gender = rStudent.getOutput().get(1).toSlot(); + NamedExpression age = rStudent.getOutput().get(3).toSlot(); + + List pkList1 = ImmutableList.of(id, gender); + List pkList2 = ImmutableList.of(gender, id, gender); + WindowFrame windowFrame = new WindowFrame(WindowFrame.FrameUnitsType.RANGE, + WindowFrame.FrameBoundary.newPrecedingBoundary(), + WindowFrame.FrameBoundary.newCurrentRowBoundary()); + WindowExpression window1 = new WindowExpression(new Rank(), pkList1, ImmutableList.of(), windowFrame); + WindowExpression window2 = new WindowExpression(new Sum(age), pkList2, ImmutableList.of(), windowFrame); + Alias windowAlias1 = new Alias(window1, window1.toSql()); + Alias windowAlias2 = new Alias(window2, window2.toSql()); + + List expressions = Lists.newArrayList(windowAlias1, windowAlias2); + Plan root = new LogicalWindow<>(expressions, rStudent).withChecked(expressions, rStudent); + + PlanChecker.from(MemoTestUtils.createConnectContext(), root) + .applyImplementation(new LogicalWindowToPhysicalWindow().build()) + .matches( + physicalWindow() + .when(physicalWindow -> { + WindowFrameGroup wfg = physicalWindow.getWindowFrameGroup(); + List windows = wfg.getGroups(); + return windows.get(0).equals(windowAlias1) && windows.get(1).equals(windowAlias2) + && CollectionUtils.isEqualCollection(wfg.getPartitionKeys(), pkList1) + && wfg.getWindowFrame().equals(windowFrame); + }) + ); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java index a23aab5d6c0e49..04302e1c62dca1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/PlannerTest.java @@ -708,7 +708,5 @@ public void testEliminatingSortNode() throws Exception { Assertions.assertFalse(plan1.contains("SORT INFO:")); Assertions.assertFalse(plan1.contains("SORT LIMIT:")); } - - } } diff --git a/regression-test/data/nereids_syntax_p0/window_function.out b/regression-test/data/nereids_syntax_p0/window_function.out new file mode 100644 index 00000000000000..70d9973a60c7ad --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/window_function.out @@ -0,0 +1,359 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !empty_over -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !with_star -- +1 \N 3.0 1 +1 1 1.0 1 +1 1 2.0 1 +1 2 \N 1 +1 2 1.0 3 +1 2 2.0 3 +1 3 \N 2 +1 3 1.0 5 +2 \N 3.0 1 +2 1 1.0 1 +2 1 2.0 1 +2 2 1.0 3 +2 2 2.0 3 +2 3 1.0 5 + +-- !multi_window_1 -- +1 1 +1 10 +1 11 +1 12 +1 13 +1 14 +1 2 +1 3 +1 4 +1 5 +1 6 +1 7 +1 8 +1 9 + +-- !multi_window_2 -- +20 12 1 +20 12 1 +20 12 1 +20 12 1 +20 12 1 +20 12 1 +20 12 1 +20 12 2 +20 12 3 +20 12 3 +20 12 3 +20 12 3 +20 12 5 +20 12 5 + +-- !dense_rank -- +1 +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +2 +3 +3 + +-- !ntile -- +1 +1 +1 +1 +1 +2 +2 +2 +2 +3 +3 +4 +4 +5 + +-- !rank -- +1 +1 +1 +1 +1 +1 +1 +2 +3 +3 +3 +3 +5 +5 + +-- !row_number -- +1 +1 +1 +1 +2 +2 +2 +2 +3 +3 +4 +4 +5 +6 + +-- !sum -- +1 +2 +3 +3 +3 +3 +3 +3 +6 +6 +6 +6 +9 +9 + +-- !avg -- +1.0 +1.0 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 +1.5 + +-- !count -- +1 +2 +2 +2 +2 +2 +2 +2 +4 +4 +4 +4 +6 +6 + +-- !first_value -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 + +-- !last_value -- +1 +1 +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !lead -- +1 +1 +1 +1 +1 +111 +111 +111 +111 +2 +2 +2 +2 +2 + +-- !lag -- +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +222 +222 +222 +222 + +-- !max -- +1 +1 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 + +-- !min -- +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + +-- !agg_1 -- +2 \N 1 1 +2 1 2 1 +2 2 3 1 +2 3 2 1 +3 \N 2 3 +3 1 4 3 +3 2 4 3 +3 3 2 3 + +-- !agg_2 -- +1 \N 2 2 +1 1 4 2 +1 2 6 2 +1 3 4 2 +2 \N 3 5 +2 1 6 5 +2 2 6 5 +2 3 3 5 + +-- !agg_3 -- +1 +1 +1 +2 +2 +3 +3 +4 +4 +5 +6 +7 +8 + +-- !subquery_1 -- +1 2 \N 1 1 +1 3 \N 1 2 +1 1 1.0 1 3 +1 2 1.0 2 4 +1 3 1.0 2 5 +1 1 2.0 3 6 +1 2 2.0 4 7 +1 \N 3.0 1 8 +2 1 1.0 2 1 +2 2 1.0 3 2 +2 3 1.0 3 3 +2 1 2.0 4 4 +2 2 2.0 5 5 +2 \N 3.0 2 6 + +-- !subquery_2 -- +10 4 6 +10 4 6 +12 5 7 +12 5 7 +2 1 1 +2 1 1 +5 2 3 +5 2 3 +5 2 3 +5 2 3 +7 3 4 +7 3 4 +7 3 4 +7 3 4 + +-- !cte -- +1 8 +1 8 +1 8 +1 8 +1 8 +1 8 +1 8 +1 8 +2 20 +2 20 +2 20 +2 20 +2 20 +2 20 + +-- !window_use_agg -- +20 + diff --git a/regression-test/suites/nereids_syntax_p0/window_function.groovy b/regression-test/suites/nereids_syntax_p0/window_function.groovy new file mode 100644 index 00000000000000..33bef9260b7075 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/window_function.groovy @@ -0,0 +1,119 @@ +// 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("test_window_function") { + sql "SET enable_nereids_planner=true" + + sql "DROP TABLE IF EXISTS window_test" + + sql """ + CREATE TABLE `window_test` ( + `c1` int NULL, + `c2` int NULL, + `c3` double NULL + ) ENGINE=OLAP + DISTRIBUTED BY HASH(`c1`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """INSERT INTO window_test VALUES(1, 1, 1)""" + sql """INSERT INTO window_test VALUES(1, 2, 1)""" + sql """INSERT INTO window_test VALUES(1, 3, 1)""" + sql """INSERT INTO window_test VALUES(2, 1, 1)""" + sql """INSERT INTO window_test VALUES(2, 2, 1)""" + sql """INSERT INTO window_test VALUES(2, 3, 1)""" + sql """INSERT INTO window_test VALUES(1, 1, 2)""" + sql """INSERT INTO window_test VALUES(1, 2, 2)""" + sql """INSERT INTO window_test VALUES(2, 1, 2)""" + sql """INSERT INTO window_test VALUES(2, 2, 2)""" + sql """INSERT INTO window_test VALUES(1, 2, null)""" + sql """INSERT INTO window_test VALUES(1, 3, null)""" + sql """INSERT INTO window_test VALUES(1, null, 3)""" + sql """INSERT INTO window_test VALUES(2, null, 3)""" + + sql "SET enable_fallback_to_original_planner=false" + + order_qt_empty_over "SELECT rank() over() FROM window_test" + order_qt_with_star "SELECT *, rank() over(partition by c3 order by c2) FROM window_test" + order_qt_multi_window_1 "SELECT rank() over(), row_number() over() FROM window_test" + order_qt_multi_window_2 """ + SELECT sum(c1) over(), count(c2) over(), rank() over(partition by c3 order by c2) + FROM window_test + """ + + order_qt_dense_rank "SELECT dense_rank() over(partition by c3 order by c2) FROM window_test" + order_qt_ntile "SELECT ntile(5) over(partition by c3 order by c2) FROM window_test" + order_qt_rank "SELECT rank() over(partition by c3 order by c2) FROM window_test" + order_qt_row_number "SELECT row_number() over(partition by c3 order by c2) FROM window_test" + order_qt_sum "SELECT sum(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_avg "SELECT avg(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_count "SELECT count(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_first_value "SELECT first_value(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_last_value "SELECT last_value(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_lead "SELECT lead(c1, 1, 111) over(partition by c3 order by c2) FROM window_test" + order_qt_lag "SELECT lag(c1, 1, 222) over(partition by c3 order by c2) FROM window_test" + order_qt_max "SELECT max(c1) over(partition by c3 order by c2) FROM window_test" + order_qt_min "SELECT min(c1) over(partition by c3 order by c2) FROM window_test" + + order_qt_agg_1 """ + SELECT c1+1, c2, sum(c1), sum(c1) over(partition by c2 order by c1) + FROM window_test + GROUP BY c1, c2 + """ + + order_qt_agg_2 """ + SELECT c1, c2, sum(c1+1), sum(c1+1) over(partition by c2 order by c1+1) + FROM window_test + GROUP BY c1, c2 + """ + + order_qt_agg_3 """ + SELECT row_number() over(partition by (grouping(c2) + grouping(c3))) + FROM window_test + GROUP BY ROLLUP (c2, c3) + """ + + qt_subquery_1 """ + SELECT *, row_number() over(partition by c1 order by c3, c2, c1, r1) as r2 + FROM ( + SELECT *, row_number() over(partition by c2 order by c3, c2, c1) as r1 + FROM window_test + ) t + """ + + order_qt_subquery_2 """ + SELECT * + FROM ( + SELECT + count(1) OVER (PARTITION BY c2 ORDER BY c3) + sum(c1) OVER (PARTITION BY c2 ORDER BY c3) AS total, + count(1) OVER (PARTITION BY c2 ORDER BY c3) AS fourcount, + sum(c1) OVER (PARTITION BY c2 ORDER BY c3) AS twosum + FROM window_test + ) sub + """ + + order_qt_cte """ + WITH cte as (select c1 as x from window_test) + SELECT x, (sum(x) over (ORDER BY x range between UNBOUNDED preceding and CURRENT ROW)) FROM cte; + """ + + order_qt_window_use_agg """ + SELECT sum(sum(c1)) over(partition by avg(c2)) + FROM window_test + """ +}