You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ja...@apache.org on 2023/01/03 04:21:21 UTC

[doris] branch master updated: [feature](Nereids): Support any_value/any function. (#15450)

This is an automated email from the ASF dual-hosted git repository.

jakevin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new ada72b055f [feature](Nereids): Support any_value/any function. (#15450)
ada72b055f is described below

commit ada72b055f3a11f14df669464eca1b140cd2904d
Author: jakevin <ja...@gmail.com>
AuthorDate: Tue Jan 3 12:21:13 2023 +0800

    [feature](Nereids): Support any_value/any function. (#15450)
---
 .../doris/catalog/BuiltinAggregateFunctions.java   |  4 +-
 .../trees/expressions/functions/agg/AnyValue.java  | 73 ++++++++++++++++++++++
 .../visitor/AggregateFunctionVisitor.java          |  5 ++
 .../suites/nereids_syntax_p0/any_value.groovy      | 26 ++++++++
 4 files changed, 107 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
index c548eb7527..4a5e9c3607 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinAggregateFunctions.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.catalog;
 
+import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue;
 import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxCountDistinct;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapIntersect;
@@ -66,7 +67,8 @@ public class BuiltinAggregateFunctions implements FunctionHelper {
             agg(Ndv.class),
             agg(Sum.class),
             agg(Variance.class, "variance", "variance_pop", "var_pop"),
-            agg(VarianceSamp.class, "variance_samp", "var_samp")
+            agg(VarianceSamp.class, "variance_samp", "var_samp"),
+            agg(AnyValue.class, "any_value", "any")
     );
 
     public static final BuiltinAggregateFunctions INSTANCE = new BuiltinAggregateFunctions();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java
new file mode 100644
index 0000000000..97ed47c86c
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AnyValue.java
@@ -0,0 +1,73 @@
+// 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.agg;
+
+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.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DataType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * any_value agg function.
+ */
+public class AnyValue extends AggregateFunction implements UnaryExpression, PropagateNullable, CustomSignature {
+
+    public AnyValue(Expression child) {
+        super("any_value", child);
+    }
+
+    public AnyValue(boolean isDistinct, Expression arg) {
+        super("any_value", false, arg);
+    }
+
+    @Override
+    public FunctionSignature customSignature() {
+        DataType dataType = getArgument(0).getDataType();
+        return FunctionSignature.ret(dataType).args(dataType);
+    }
+
+    @Override
+    protected List<DataType> intermediateTypes() {
+        return ImmutableList.of(getDataType());
+    }
+
+    @Override
+    public AnyValue withDistinctAndChildren(boolean isDistinct, List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new AnyValue(isDistinct, children.get(0));
+    }
+
+    @Override
+    public AnyValue withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new AnyValue(children.get(0));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitAnyValue(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
index 0714c42dc4..ee2403e9e2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/AggregateFunctionVisitor.java
@@ -18,6 +18,7 @@
 package org.apache.doris.nereids.trees.expressions.visitor;
 
 import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
+import org.apache.doris.nereids.trees.expressions.functions.agg.AnyValue;
 import org.apache.doris.nereids.trees.expressions.functions.agg.ApproxCountDistinct;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Avg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount;
@@ -109,4 +110,8 @@ public interface AggregateFunctionVisitor<R, C> {
     default R visitApproxCountDistinct(ApproxCountDistinct approxCountDistinct, C context) {
         return visitAggregateFunction(approxCountDistinct, context);
     }
+
+    default R visitAnyValue(AnyValue anyValue, C context) {
+        return visitAggregateFunction(anyValue, context);
+    }
 }
diff --git a/regression-test/suites/nereids_syntax_p0/any_value.groovy b/regression-test/suites/nereids_syntax_p0/any_value.groovy
new file mode 100644
index 0000000000..3de2b0dd73
--- /dev/null
+++ b/regression-test/suites/nereids_syntax_p0/any_value.groovy
@@ -0,0 +1,26 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("any_value") {
+    // enable nereids
+    sql "SET enable_nereids_planner=true"
+    sql "SET enable_fallback_to_original_planner=false"
+
+    test {
+        sql "select any(s_suppkey), any(s_name), any_value(s_address) from supplier;"
+    }
+}
\ No newline at end of file


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