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

zclll 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 8192bbfa384 [Feature](agg) Support bool agg functions (#55643)
8192bbfa384 is described below

commit 8192bbfa3844896823955a03fc6a860c124995b5
Author: linrrarity <[email protected]>
AuthorDate: Sun Sep 7 18:17:29 2025 +0800

    [Feature](agg) Support bool agg functions (#55643)
    
    Support `BOOL_AND(BOOLAND_AGG)`, `BOOL_OR(BOOLOR_AGG)`,
    `BOOL_XOR(BOOLXOR_AGG)`
---
 .../aggregate_function_bool_union.cpp              |  44 ++++++++
 .../aggregate_function_bool_union.h                | 111 +++++++++++++++++++++
 .../aggregate_function_simple_factory.cpp          |   2 +
 .../aggregate_functions/agg_bool_union_test.cpp    |  74 ++++++++++++++
 .../doris/catalog/BuiltinAggregateFunctions.java   |   6 ++
 .../trees/expressions/functions/agg/BoolAnd.java   |  90 +++++++++++++++++
 .../trees/expressions/functions/agg/BoolOr.java    |  86 ++++++++++++++++
 .../trees/expressions/functions/agg/BoolXor.java   |  86 ++++++++++++++++
 .../functions/executable/NumericArithmetic.java    |  24 +++++
 .../visitor/AggregateFunctionVisitor.java          |  15 +++
 .../test_aggregate_all_functions2.out              | Bin 8671 -> 9232 bytes
 .../test_aggregate_all_functions2.groovy           |  21 ++++
 12 files changed, 559 insertions(+)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_bool_union.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_bool_union.cpp
new file mode 100644
index 00000000000..a05fb0b1902
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_bool_union.cpp
@@ -0,0 +1,44 @@
+// 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.
+
+#include "vec/aggregate_functions/aggregate_function_bool_union.h"
+
+#include <fmt/format.h>
+
+#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
+#include "vec/aggregate_functions/helpers.h"
+#include "vec/data_types/data_type.h"
+
+namespace doris::vectorized {
+#include "common/compile_check_begin.h"
+
+void register_aggregate_function_bool_union(AggregateFunctionSimpleFactory& 
factory) {
+    factory.register_function_both(
+            "bool_or", creator_with_type_list<TYPE_BOOLEAN>::template creator<
+                               AggregateFunctionBitwise, 
AggregateFunctionGroupBitOrData>);
+    factory.register_function_both(
+            "bool_and", creator_with_type_list<TYPE_BOOLEAN>::template creator<
+                                AggregateFunctionBitwise, 
AggregateFunctionGroupBitAndData>);
+    factory.register_function_both(
+            "bool_xor",
+            
creator_without_type::creator<AggregateFuntionBoolUnion<AggregateFunctionBoolXorData>>);
+    factory.register_alias("bool_or", "boolor_agg");
+    factory.register_alias("bool_and", "booland_agg");
+    factory.register_alias("bool_xor", "boolxor_agg");
+}
+} // namespace doris::vectorized
+#include "common/compile_check_end.h"
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_bool_union.h 
b/be/src/vec/aggregate_functions/aggregate_function_bool_union.h
new file mode 100644
index 00000000000..d0710cc071e
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_bool_union.h
@@ -0,0 +1,111 @@
+// 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.
+
+#pragma once
+
+#include "runtime/define_primitive_type.h"
+#include "runtime/primitive_type.h"
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_bit.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/columns/column_vector.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_nullable.h"
+#include "vec/data_types/data_type_number.h"
+
+namespace doris::vectorized {
+#include "common/compile_check_begin.h"
+
+struct AggregateFunctionBoolXorData {
+    static constexpr auto name = "bool_xor";
+
+    void add(bool x) {
+        if (x && _st < 2) {
+            ++_st;
+        }
+    }
+
+    void merge(const AggregateFunctionBoolXorData& rhs) {
+        if (_st == 0) {
+            _st = rhs._st;
+        } else if (_st == 1) {
+            _st = (rhs._st > 0) ? 2 : 1;
+        }
+    }
+
+    void write(BufferWritable& buf) const { buf.write_binary(_st); }
+
+    void read(BufferReadable& buf) { buf.read_binary(_st); }
+
+    void reset() { _st = 0; }
+
+    bool get() const { return _st == 1; }
+
+private:
+    // represents the current XOR state
+    // '0': there are no true values currently
+    // '1': exactly one true value has appeared
+    // '2': two true values have already appeared and will not change 
thereafter
+    uint8_t _st = 0;
+};
+
+template <typename BoolFunc>
+class AggregateFuntionBoolUnion final
+        : public IAggregateFunctionDataHelper<BoolFunc, 
AggregateFuntionBoolUnion<BoolFunc>>,
+          NullableAggregateFunction,
+          UnaryExpression {
+public:
+    explicit AggregateFuntionBoolUnion(const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<BoolFunc, 
AggregateFuntionBoolUnion<BoolFunc>>(
+                      argument_types_) {}
+
+    String get_name() const override { return BoolFunc::name; }
+
+    DataTypePtr get_return_type() const override { return 
std::make_shared<DataTypeBool>(); }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
+             Arena&) const override {
+        this->data(place).add(
+                assert_cast<const ColumnUInt8&, 
TypeCheckOnRelease::DISABLE>(*columns[0])
+                        .get_element(row_num));
+    }
+
+    void reset(AggregateDataPtr __restrict place) const override { 
this->data(place).reset(); }
+
+    void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs,
+               Arena&) const override {
+        this->data(place).merge(this->data(rhs));
+    }
+
+    void serialize(ConstAggregateDataPtr __restrict place, BufferWritable& 
buf) const override {
+        this->data(place).write(buf);
+    }
+
+    void deserialize(AggregateDataPtr __restrict place, BufferReadable& buf,
+                     Arena&) const override {
+        this->data(place).read(buf);
+    }
+
+    void insert_result_into(ConstAggregateDataPtr __restrict place, IColumn& 
to) const override {
+        assert_cast<ColumnUInt8&>(to).insert_value(this->data(place).get());
+    }
+};
+} // namespace doris::vectorized
+
+#include "common/compile_check_end.h"
\ No newline at end of file
diff --git 
a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
index 732da05fd36..50123ad2111 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
@@ -80,6 +80,7 @@ void 
register_aggregate_function_approx_top_k(AggregateFunctionSimpleFactory& fa
 void 
register_aggregate_function_approx_top_sum(AggregateFunctionSimpleFactory& 
factory);
 void 
register_aggregate_function_percentile_reservoir(AggregateFunctionSimpleFactory&
 factory);
 void register_aggregate_function_ai_agg(AggregateFunctionSimpleFactory& 
factory);
+void register_aggregate_function_bool_union(AggregateFunctionSimpleFactory& 
factory);
 
 AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() {
     static std::once_flag oc;
@@ -137,6 +138,7 @@ AggregateFunctionSimpleFactory& 
AggregateFunctionSimpleFactory::instance() {
         register_aggregate_function_approx_top_sum(instance);
         register_aggregate_function_percentile_reservoir(instance);
         register_aggregate_function_ai_agg(instance);
+        register_aggregate_function_bool_union(instance);
         // Register foreach and foreachv2 functions
         register_aggregate_function_combinator_foreach(instance);
         register_aggregate_function_combinator_foreachv2(instance);
diff --git a/be/test/vec/aggregate_functions/agg_bool_union_test.cpp 
b/be/test/vec/aggregate_functions/agg_bool_union_test.cpp
new file mode 100644
index 00000000000..02dd922ec9e
--- /dev/null
+++ b/be/test/vec/aggregate_functions/agg_bool_union_test.cpp
@@ -0,0 +1,74 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include "agg_function_test.h"
+#include "vec/data_types/data_type_number.h"
+
+namespace doris::vectorized {
+
+struct AggregateFunctionBoolUnionTest : public AggregateFunctiontest {};
+
+TEST_F(AggregateFunctionBoolUnionTest, test_bool_or) {
+    {
+        create_agg("bool_or", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({false, 
false, true})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({true}));
+    }
+
+    {
+        create_agg("boolor_agg", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({true, 
false, true})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({true}));
+    }
+}
+
+TEST_F(AggregateFunctionBoolUnionTest, test_bool_and) {
+    {
+        create_agg("bool_and", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({true, 
true})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({true}));
+    }
+
+    {
+        create_agg("booland_agg", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({true, 
false, true})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({false}));
+    }
+}
+
+TEST_F(AggregateFunctionBoolUnionTest, test_bool_xor) {
+    {
+        create_agg("bool_xor", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({true, true, 
true})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({false}));
+    }
+
+    {
+        create_agg("boolxor_agg", false, {std::make_shared<DataTypeBool>()});
+
+        
execute(Block({ColumnHelper::create_column_with_name<DataTypeBool>({true, 
false, false})}),
+                ColumnHelper::create_column_with_name<DataTypeBool>({true}));
+    }
+}
+} // namespace doris::vectorized
\ No newline at end of file
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 4a173eef588..831776731ae 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
@@ -29,6 +29,9 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.BitmapIntersect;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnion;
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionInt;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolAnd;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolOr;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolXor;
 import org.apache.doris.nereids.trees.expressions.functions.agg.CollectList;
 import org.apache.doris.nereids.trees.expressions.functions.agg.CollectSet;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Corr;
@@ -117,6 +120,9 @@ public class BuiltinAggregateFunctions implements 
FunctionHelper {
             agg(BitmapUnion.class, "bitmap_union"),
             agg(BitmapUnionCount.class, "bitmap_union_count"),
             agg(BitmapUnionInt.class, "bitmap_union_int"),
+            agg(BoolOr.class, "bool_or", "boolor_agg"),
+            agg(BoolAnd.class, "bool_and", "booland_agg"),
+            agg(BoolXor.class, "bool_xor", "boolxor_agg"),
             agg(CollectList.class, "collect_list", "group_array"),
             agg(CollectSet.class, "collect_set", "group_uniq_array"),
             agg(Corr.class, "corr"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolAnd.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolAnd.java
new file mode 100644
index 00000000000..b686d165203
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolAnd.java
@@ -0,0 +1,90 @@
+// 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.exceptions.AnalysisException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.types.DataType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * AggregateFunction 'bool_and'.
+ */
+public class BoolAnd extends NullableAggregateFunction
+        implements UnaryExpression, ExplicitlyCastableSignature {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(BooleanType.INSTANCE).args(BooleanType.INSTANCE)
+    );
+
+    public BoolAnd(Expression child) {
+        this(false, false, child);
+    }
+
+    private BoolAnd(boolean distinct, Expression arg) {
+        this(distinct, false, arg);
+    }
+
+    private BoolAnd(boolean distinct, boolean alwaysNullable, Expression arg) {
+        super("bool_and", distinct, alwaysNullable, arg);
+    }
+
+    /**
+     * constructor for withChildren and reuse signature
+     */
+    private BoolAnd(NullableAggregateFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    @Override
+    public BoolAnd withDistinctAndChildren(boolean distinct, List<Expression> 
children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new BoolAnd(getFunctionParams(distinct, children));
+    }
+
+    @Override
+    public void checkLegalityBeforeTypeCoercion() {
+        DataType argType = child().getDataType();
+        if (!(argType.isBooleanType() || argType.isNumericType())) {
+            throw new AnalysisException("bool_and requires a boolean or 
numeric argument");
+        }
+    }
+
+    @Override
+    public NullableAggregateFunction withAlwaysNullable(boolean 
alwaysNullable) {
+        return new BoolAnd(getAlwaysNullableFunctionParams(alwaysNullable));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitBoolAnd(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolOr.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolOr.java
new file mode 100644
index 00000000000..3e4f393b226
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolOr.java
@@ -0,0 +1,86 @@
+// 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.exceptions.AnalysisException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.types.DataType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * AggregateFunction 'bool_or'.
+ */
+public class BoolOr extends NullableAggregateFunction
+        implements UnaryExpression, ExplicitlyCastableSignature {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(BooleanType.INSTANCE).args(BooleanType.INSTANCE)
+    );
+
+    public BoolOr(Expression child) {
+        this(false, false, child);
+    }
+
+    private BoolOr(boolean distinct, boolean alwaysNullable, Expression arg) {
+        super("bool_or", distinct, alwaysNullable, arg);
+    }
+
+    /**
+     * constructor for withChildren and reuse signature
+     */
+    private BoolOr(NullableAggregateFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    @Override
+    public BoolOr withDistinctAndChildren(boolean distinct, List<Expression> 
children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new BoolOr(getFunctionParams(distinct, children));
+    }
+
+    @Override
+    public void checkLegalityBeforeTypeCoercion() {
+        DataType argType = child().getDataType();
+        if (!(argType.isBooleanType() || argType.isNumericType())) {
+            throw new AnalysisException("bool_or requires a boolean or numeric 
argument");
+        }
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitBoolOr(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public NullableAggregateFunction withAlwaysNullable(boolean 
alwaysNullable) {
+        return new BoolOr(getAlwaysNullableFunctionParams(alwaysNullable));
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolXor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolXor.java
new file mode 100644
index 00000000000..4bdee9c22cb
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/BoolXor.java
@@ -0,0 +1,86 @@
+// 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.exceptions.AnalysisException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.types.DataType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * AggregateFunction 'bool_xor'.
+ */
+public class BoolXor extends NullableAggregateFunction
+        implements UnaryExpression, ExplicitlyCastableSignature {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(BooleanType.INSTANCE).args(BooleanType.INSTANCE)
+    );
+
+    public BoolXor(Expression child) {
+        this(false, false, child);
+    }
+
+    private BoolXor(boolean distinct, boolean alwaysNullable, Expression arg) {
+        super("bool_xor", distinct, alwaysNullable, arg);
+    }
+
+    /**
+     * constructor for withChildren and reuse signature
+     */
+    private BoolXor(NullableAggregateFunctionParams functionParams) {
+        super(functionParams);
+    }
+
+    @Override
+    public BoolXor withDistinctAndChildren(boolean distinct, List<Expression> 
children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new BoolXor(getFunctionParams(distinct, children));
+    }
+
+    @Override
+    public void checkLegalityBeforeTypeCoercion() {
+        DataType argType = child().getDataType();
+        if (!(argType.isBooleanType() || argType.isNumericType())) {
+            throw new AnalysisException("bool_xor requires a boolean or 
numeric argument");
+        }
+    }
+
+    @Override
+    public NullableAggregateFunction withAlwaysNullable(boolean 
alwaysNullable) {
+        return new BoolXor(getAlwaysNullableFunctionParams(alwaysNullable));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitBoolXor(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java
index aff700fa4c4..5aa5aa831f7 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/NumericArithmetic.java
@@ -1037,4 +1037,28 @@ public class NumericArithmetic {
     public static Expression isinf(FloatLiteral first) {
         return BooleanLiteral.of(Float.isInfinite(first.getValue()));
     }
+
+    /**
+     * bool_and
+     */
+    @ExecFunction(name = "bool_and")
+    public static Expression booland(BooleanLiteral first) {
+        return first;
+    }
+
+    /**
+     * bool_or
+     */
+    @ExecFunction(name = "bool_or")
+    public static Expression boolor(BooleanLiteral first) {
+        return first;
+    }
+
+    /**
+     * bool_xor
+     */
+    @ExecFunction(name = "bool_xor")
+    public static Expression boolxor(BooleanLiteral first) {
+        return first;
+    }
 }
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 1aea14fb2e3..6afa20ec29a 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
@@ -30,6 +30,9 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.BitmapIntersect;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnion;
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount;
 import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionInt;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolAnd;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolOr;
+import org.apache.doris.nereids.trees.expressions.functions.agg.BoolXor;
 import org.apache.doris.nereids.trees.expressions.functions.agg.CollectList;
 import org.apache.doris.nereids.trees.expressions.functions.agg.CollectSet;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Corr;
@@ -137,6 +140,18 @@ public interface AggregateFunctionVisitor<R, C> {
         return visitAggregateFunction(bitmapUnionInt, context);
     }
 
+    default R visitBoolAnd(BoolAnd boolAnd, C context) {
+        return visitAggregateFunction(boolAnd, context);
+    }
+
+    default R visitBoolOr(BoolOr boolOr, C context) {
+        return visitAggregateFunction(boolOr, context);
+    }
+
+    default R visitBoolXor(BoolXor boolXor, C context) {
+        return visitAggregateFunction(boolXor, context);
+    }
+
     default R visitCollectList(CollectList collectList, C context) {
         return visitAggregateFunction(collectList, context);
     }
diff --git 
a/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out
 
b/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out
index fa8ed377ff7..05311a69b94 100644
Binary files 
a/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out
 and 
b/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out
 differ
diff --git 
a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy
 
b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy
index d4ff2511ead..2bcea32cb94 100644
--- 
a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy
@@ -103,6 +103,27 @@ suite("test_aggregate_all_functions2") {
         exception "percentile_reservoir level must be in [0, 1]"
     }
 
+    qt_bool_and """SELECT bool_and(k0) FROM baseall;"""
+    qt_bool_and """SELECT bool_and(k1) FROM baseall;"""
+    qt_bool_and """SELECT bool_and(k8) over(partition by k6) from baseall 
order by k1;"""
+    qt_bool_and """SELECT booland_agg(k0) over(partition by k8) from baseall 
order by k3;"""
+    qt_bool_or """SELECT bool_or(k2) FROM baseall group by k6 order by 1;"""
+    qt_bool_or """SELECT bool_or(k3) FROM baseall;"""
+    qt_bool_or """SELECT boolor_agg(k4) over(partition by k9) from baseall 
order by k7;"""
+    qt_bool_xor """SELECT bool_xor(k4) FROM baseall;"""
+    qt_bool_xor """SELECT bool_xor(k5) FROM baseall group by k6 order by 1;"""
+    qt_bool_xor """SELECT boolxor_agg(k13) over(partition by k10) from baseall 
order by k2;"""
+    
+    test {
+        sql """SELECT booland_agg(k7) FROM baseall;"""
+        exception "requires a boolean or numeric argument"
+    }
+
+    test {
+        sql """SELECT boolor_agg(k10) FROM baseall;"""
+        exception "requires a boolean or numeric argument"
+    }
+
     sql "DROP DATABASE IF EXISTS metric_table"
     sql """
         CREATE TABLE `metric_table` (


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to