snuyanzin commented on code in PR #26547:
URL: https://github.com/apache/flink/pull/26547#discussion_r2122232666


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.functions.DeclarativeAggregateFunction;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.fun.SqlLiteralAggFunction;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+
+/**
+ * Built-in literal aggregate function. This function is used for internal 
optimizations. It accepts
+ * zero regular aggregate arguments and returns a constant value. For more 
details see <a
+ * href="https://issues.apache.org/jira/browse/CALCITE-4334";>CALCITE-4334</a> 
and {@link
+ * SqlLiteralAggFunction}.
+ */
+public abstract class LiteralAggFunction extends DeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression literalAgg = 
unresolvedRef("literalAgg");
+    private final RexLiteral rexLiteral;
+
+    public LiteralAggFunction(RexLiteral rexLiteral) {
+        this.rexLiteral = rexLiteral;
+    }
+
+    @Override
+    public int operandCount() {
+        return 0;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {literalAgg};

Review Comment:
   This method still stays, however all others now return empty collection.
   Especially `initialValuesExpressions` which is participated in decision: 
create/not create accumulator 
https://github.com/apache/flink/blob/cf55d306f570134e2d3d332a0acba7d8737c87b7/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/agg/DeclarativeAggCodeGen.scala#L105-L108



-- 
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: issues-unsubscr...@flink.apache.org

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

Reply via email to