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]