morrySnow commented on code in PR #14397:
URL: https://github.com/apache/doris/pull/14397#discussion_r1091595083


##########
fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4:
##########
@@ -327,6 +332,32 @@ functionIdentifier
     | LEFT | RIGHT
     ;
 
+windowSpec
+    : name=identifier
+    | LEFT_PAREN name=identifier RIGHT_PAREN
+    | LEFT_PAREN
+        partitionClause?
+        sortClause?
+        windowFrame?
+        RIGHT_PAREN

Review Comment:
   does this mean the empty parentheses is legal?
   ```
   SELECT ROW_NUMBER() OVER () FROM t;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java:
##########
@@ -182,12 +181,13 @@ 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;
             }
 
+            System.out.println(physicalPlan.treeString());
+

Review Comment:
   remove it~



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Window.java:
##########
@@ -0,0 +1,176 @@
+// 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.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+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.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 Window extends Expression implements UnaryExpression, 
PropagateNullable {
+
+    private Optional<List<Expression>> partitionKeyList;
+
+    private Optional<List<OrderKey>> orderKeyList;
+
+    private Optional<WindowFrame> windowFrame;

Review Comment:
   final



##########
fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4:
##########
@@ -327,6 +332,32 @@ functionIdentifier
     | LEFT | RIGHT
     ;
 
+windowSpec
+    : name=identifier
+    | LEFT_PAREN name=identifier RIGHT_PAREN
+    | LEFT_PAREN
+        partitionClause?
+        sortClause?
+        windowFrame?
+        RIGHT_PAREN
+    ;
+
+windowFrame
+    : frameUnits start=frameBound
+    | frameUnits BETWEEN start=frameBound AND end=frameBound
+    ;
+
+frameUnits
+    : ROWS
+    | RANGE
+    ;
+
+frameBound

Review Comment:
   ```suggestion
   frameBoundary
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Window.java:
##########
@@ -0,0 +1,176 @@
+// 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.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+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.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 Window extends Expression implements UnaryExpression, 
PropagateNullable {

Review Comment:
   i think we should treat all window spec as window expression's children. 
then we could do expression rewrite easily
   refer to spark~



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -945,15 +954,99 @@ 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) {
+                        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 
window function: " + functionName, ctx);

Review Comment:
   throw error same as legacy planner
   ```
   DISTINCT not allowed in analytic function: count(DISTINCT 1)
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Window.java:
##########
@@ -0,0 +1,176 @@
+// 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.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+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.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 Window extends Expression implements UnaryExpression, 
PropagateNullable {
+
+    private Optional<List<Expression>> partitionKeyList;

Review Comment:
   ```suggestion
       private final Optional<List<Expression>> partitionKeys;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Window.java:
##########
@@ -0,0 +1,176 @@
+// 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.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+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.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 Window extends Expression implements UnaryExpression, 
PropagateNullable {

Review Comment:
   rename to `WindowExpression` to distinguished from window plan~



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Window.java:
##########
@@ -0,0 +1,176 @@
+// 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.properties.OrderKey;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+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.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 Window extends Expression implements UnaryExpression, 
PropagateNullable {
+
+    private Optional<List<Expression>> partitionKeyList;
+
+    private Optional<List<OrderKey>> orderKeyList;

Review Comment:
   ```suggestion
       private final Optional<List<OrderKey>> orderKeys;
   ```



##########
fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4:
##########
@@ -327,6 +332,32 @@ functionIdentifier
     | LEFT | RIGHT
     ;
 
+windowSpec
+    : name=identifier
+    | LEFT_PAREN name=identifier RIGHT_PAREN

Review Comment:
   does legacy planner support windowRef?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -945,15 +954,99 @@ 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) {
+                        throw new ParseException(
+                                "COUNT(*) isn't supported as window function; 
can use COUNT(col)", ctx);
+                    }

Review Comment:
   add a todo, after we remove legacy planner, we should support count(*) as 
window function~



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java:
##########
@@ -0,0 +1,200 @@
+// 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;
+import java.util.stream.Collectors;
+
+/**
+ * logical node to deal with window functions
+ */
+public class LogicalWindow<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> implements Window {
+
+    private List<NamedExpression> outputExpressions;
+
+    private List<NamedExpression> windowExpressions;
+
+    private boolean isNormalized;
+
+    private boolean isChecked;

Review Comment:
   final



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java:
##########
@@ -0,0 +1,200 @@
+// 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;
+import java.util.stream.Collectors;
+
+/**
+ * logical node to deal with window functions
+ */
+public class LogicalWindow<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> implements Window {
+
+    private List<NamedExpression> outputExpressions;
+
+    private List<NamedExpression> windowExpressions;
+
+    private boolean isNormalized;
+
+    private boolean isChecked;
+
+    /**
+     * used in step Analyze
+     */
+    public LogicalWindow(List<NamedExpression> outputExpressions, CHILD_TYPE 
child) {
+        this(outputExpressions, null, false, false, Optional.empty(), 
Optional.empty(), child);
+    }
+
+    public LogicalWindow(List<NamedExpression> outputExpressions, 
List<NamedExpression> windowExpressions,
+                         boolean isNormalized, boolean isChecked, CHILD_TYPE 
child) {
+        this(outputExpressions, windowExpressions, isNormalized, isChecked, 
Optional.empty(),
+                Optional.empty(), child);
+    }
+
+    public LogicalWindow(List<NamedExpression> outputExpressions, 
List<NamedExpression> windowExpressions,
+                         boolean isNormalized, boolean isChecked,
+                         Optional<GroupExpression> groupExpression, 
Optional<LogicalProperties> logicalProperties,
+                         CHILD_TYPE child) {
+        super(PlanType.LOGICAL_WINDOW, groupExpression, logicalProperties, 
child);
+        this.outputExpressions = outputExpressions;
+        this.windowExpressions = windowExpressions;

Review Comment:
   ImmutableList.copyOf(Objects.requireNotNull(..., "...."))



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java:
##########
@@ -0,0 +1,200 @@
+// 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;
+import java.util.stream.Collectors;
+
+/**
+ * logical node to deal with window functions
+ */
+public class LogicalWindow<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> implements Window {
+
+    private List<NamedExpression> outputExpressions;

Review Comment:
   why we need a explicit `outputExpressions`? i think output could get from 
`windowExpressions` by `windowExpressions.stream().map(e -> 
e.toSlot()).collect(Collectors.toList());`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java:
##########
@@ -0,0 +1,200 @@
+// 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;
+import java.util.stream.Collectors;
+
+/**
+ * logical node to deal with window functions
+ */
+public class LogicalWindow<CHILD_TYPE extends Plan> extends 
LogicalUnary<CHILD_TYPE> implements Window {
+
+    private List<NamedExpression> outputExpressions;
+
+    private List<NamedExpression> windowExpressions;
+
+    private boolean isNormalized;
+
+    private boolean isChecked;
+
+    /**
+     * used in step Analyze
+     */
+    public LogicalWindow(List<NamedExpression> outputExpressions, CHILD_TYPE 
child) {
+        this(outputExpressions, null, false, false, Optional.empty(), 
Optional.empty(), child);
+    }
+
+    public LogicalWindow(List<NamedExpression> outputExpressions, 
List<NamedExpression> windowExpressions,
+                         boolean isNormalized, boolean isChecked, CHILD_TYPE 
child) {
+        this(outputExpressions, windowExpressions, isNormalized, isChecked, 
Optional.empty(),
+                Optional.empty(), child);
+    }
+
+    public LogicalWindow(List<NamedExpression> outputExpressions, 
List<NamedExpression> windowExpressions,
+                         boolean isNormalized, boolean isChecked,
+                         Optional<GroupExpression> groupExpression, 
Optional<LogicalProperties> logicalProperties,
+                         CHILD_TYPE child) {
+        super(PlanType.LOGICAL_WINDOW, groupExpression, logicalProperties, 
child);
+        this.outputExpressions = outputExpressions;
+        this.windowExpressions = windowExpressions;
+        this.isNormalized = isNormalized;
+        this.isChecked = isChecked;
+    }
+
+    public boolean isNormalized() {
+        return isNormalized;
+    }
+
+    public boolean isChecked() {
+        return isChecked;
+    }
+
+    @Override
+    public List<NamedExpression> getOutputExpressions() {
+        return outputExpressions;
+    }
+
+    public List<NamedExpression> getWindowExpressions() {
+        return windowExpressions;
+    }
+
+    /**
+     * get all expressions used in LogicalWindow
+     */
+    @Override
+    public List<? extends Expression> getExpressions() {
+        ImmutableList.Builder<Expression> builder = new 
ImmutableList.Builder<>();
+        builder.addAll(outputExpressions);
+        if (windowExpressions != null) {

Review Comment:
   why windowExpressions could be null?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Window.java:
##########
@@ -0,0 +1,97 @@
+// 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.functions.window.FrameBoundType;
+import 
org.apache.doris.nereids.trees.expressions.functions.window.FrameBoundary;
+import 
org.apache.doris.nereids.trees.expressions.functions.window.FrameUnitsType;
+import org.apache.doris.nereids.trees.expressions.literal.Literal;
+
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * interface for LogicalWindow and PhysicalWindow
+ */
+public interface Window {
+
+    List<NamedExpression> getOutputExpressions();
+
+    default List<Expression> 
extractExpressionsFromWindow(List<NamedExpression> windowExpressions) {
+        return windowExpressions.stream().map(expression -> 
expression.child(0))
+            .map(org.apache.doris.nereids.trees.expressions.Window.class::cast)
+            .flatMap(window -> window.getExpressionsInWindowSpec().stream())
+            .collect(Collectors.toList());
+    }
+
+    /**
+     * 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);
+    }

Review Comment:
   i think it it better that put translate functions in translator 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to