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

dataroaring pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new da3ab02bdfe [feat](skew & kurt) New aggregate function skew & kurt 
#40945 (#41277)
da3ab02bdfe is described below

commit da3ab02bdfeca1dc9b3e3d94fde7caa955f1c1f4
Author: zhiqiang <seuhezhiqi...@163.com>
AuthorDate: Sat Sep 28 18:36:21 2024 +0800

    [feat](skew & kurt) New aggregate function skew & kurt #40945 (#41277)
    
    cherry pick from #40945
---
 .../pipeline/exec/aggregation_source_operator.cpp  |   5 +
 .../aggregate_function_kurtosis.cpp                |  80 ++++++++++
 .../aggregate_function_simple_factory.cpp          |   5 +
 .../aggregate_function_simple_factory.h            |   1 -
 .../aggregate_function_skew.cpp                    |  80 ++++++++++
 .../aggregate_function_statistic.h                 | 163 +++++++++++++++++++++
 be/src/vec/aggregate_functions/moments.h           | 114 ++++++++++++++
 .../doris/catalog/BuiltinAggregateFunctions.java   |   6 +-
 .../java/org/apache/doris/catalog/FunctionSet.java |  36 +++++
 .../trees/expressions/functions/agg/Kurt.java      |  79 ++++++++++
 .../trees/expressions/functions/agg/Skew.java      |  80 ++++++++++
 .../visitor/AggregateFunctionVisitor.java          |  10 ++
 .../query_p0/aggregate/aggregate_function_kurt.out |  52 +++++++
 .../query_p0/aggregate/aggregate_function_skew.out |  52 +++++++
 .../aggregate/aggregate_function_kurt.groovy       |  78 ++++++++++
 .../aggregate/aggregate_function_skew.groovy       |  78 ++++++++++
 16 files changed, 917 insertions(+), 2 deletions(-)

diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp 
b/be/src/pipeline/exec/aggregation_source_operator.cpp
index a5f40a431c5..fe03eba4102 100644
--- a/be/src/pipeline/exec/aggregation_source_operator.cpp
+++ b/be/src/pipeline/exec/aggregation_source_operator.cpp
@@ -416,6 +416,11 @@ Status 
AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B
                 }
             }
 
+            // Result of operator is nullable, but aggregate function result 
is not nullable
+            // this happens when:
+            // 1. no group by
+            // 2. input of aggregate function is empty
+            // 3. all of input columns are not nullable
             if (column_type->is_nullable() && !data_types[i]->is_nullable()) {
                 vectorized::ColumnPtr ptr = std::move(columns[i]);
                 // unless `count`, other aggregate function dispose empty set 
should be null
diff --git a/be/src/vec/aggregate_functions/aggregate_function_kurtosis.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_kurtosis.cpp
new file mode 100644
index 00000000000..00ad1893eaf
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_kurtosis.cpp
@@ -0,0 +1,80 @@
+// 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 "common/status.h"
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
+#include "vec/aggregate_functions/aggregate_function_statistic.h"
+#include "vec/aggregate_functions/helpers.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_nullable.h"
+
+namespace doris::vectorized {
+
+template <typename T>
+AggregateFunctionPtr type_dispatch_for_aggregate_function_kurt(const 
DataTypes& argument_types,
+                                                               const bool 
result_is_nullable,
+                                                               bool 
nullable_input) {
+    using StatFunctionTemplate = StatFuncOneArg<T, 4>;
+
+    if (nullable_input) {
+        return creator_without_type::create_ignore_nullable<
+                AggregateFunctionVarianceSimple<StatFunctionTemplate, true>>(
+                argument_types, result_is_nullable, 
STATISTICS_FUNCTION_KIND::KURT_POP);
+    } else {
+        return creator_without_type::create_ignore_nullable<
+                AggregateFunctionVarianceSimple<StatFunctionTemplate, false>>(
+                argument_types, result_is_nullable, 
STATISTICS_FUNCTION_KIND::KURT_POP);
+    }
+};
+
+AggregateFunctionPtr create_aggregate_function_kurt(const std::string& name,
+                                                    const DataTypes& 
argument_types,
+                                                    const bool 
result_is_nullable) {
+    if (argument_types.size() != 1) {
+        LOG(WARNING) << "aggregate function " << name << " requires exactly 1 
argument";
+        return nullptr;
+    }
+
+    if (!result_is_nullable) {
+        LOG(WARNING) << "aggregate function " << name << " requires nullable 
result type";
+        return nullptr;
+    }
+
+    const bool nullable_input = argument_types[0]->is_nullable();
+    WhichDataType type(remove_nullable(argument_types[0]));
+
+#define DISPATCH(TYPE)                                                         
                    \
+    if (type.idx == TypeIndex::TYPE)                                           
                    \
+        return type_dispatch_for_aggregate_function_kurt<TYPE>(argument_types, 
result_is_nullable, \
+                                                               nullable_input);
+    FOR_NUMERIC_TYPES(DISPATCH)
+#undef DISPATCH
+
+    LOG(WARNING) << "unsupported input type " << argument_types[0]->get_name()
+                 << " for aggregate function " << name;
+    return nullptr;
+}
+
+void register_aggregate_function_kurtosis(AggregateFunctionSimpleFactory& 
factory) {
+    factory.register_function_both("kurt", create_aggregate_function_kurt);
+    factory.register_alias("kurt", "kurt_pop");
+    factory.register_alias("kurt", "kurtosis");
+}
+
+} // namespace doris::vectorized
\ 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 436691c6ef2..d11ec714889 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.cpp
@@ -68,6 +68,8 @@ void 
register_aggregate_function_bitmap_agg(AggregateFunctionSimpleFactory& fact
 void register_aggregate_functions_corr(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_covar_pop(AggregateFunctionSimpleFactory& 
factory);
 void register_aggregate_function_covar_samp(AggregateFunctionSimpleFactory& 
factory);
+void register_aggregate_function_skewness(AggregateFunctionSimpleFactory& 
factory);
+void register_aggregate_function_kurtosis(AggregateFunctionSimpleFactory& 
factory);
 
 AggregateFunctionSimpleFactory& AggregateFunctionSimpleFactory::instance() {
     static std::once_flag oc;
@@ -119,6 +121,9 @@ AggregateFunctionSimpleFactory& 
AggregateFunctionSimpleFactory::instance() {
         register_aggregate_function_covar_samp(instance);
 
         register_aggregate_function_combinator_foreach(instance);
+
+        register_aggregate_function_skewness(instance);
+        register_aggregate_function_kurtosis(instance);
     });
     return instance;
 }
diff --git a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h 
b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
index b22504dda9c..cdd0c1a900b 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_simple_factory.h
@@ -138,7 +138,6 @@ public:
         if (function_alias.contains(name)) {
             name_str = function_alias[name];
         }
-
         if (nullable) {
             return nullable_aggregate_functions.find(name_str) == 
nullable_aggregate_functions.end()
                            ? nullptr
diff --git a/be/src/vec/aggregate_functions/aggregate_function_skew.cpp 
b/be/src/vec/aggregate_functions/aggregate_function_skew.cpp
new file mode 100644
index 00000000000..144e482ad23
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_skew.cpp
@@ -0,0 +1,80 @@
+// 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 "common/status.h"
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/aggregate_function_simple_factory.h"
+#include "vec/aggregate_functions/aggregate_function_statistic.h"
+#include "vec/aggregate_functions/helpers.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type.h"
+#include "vec/data_types/data_type_nullable.h"
+
+namespace doris::vectorized {
+
+template <typename T>
+AggregateFunctionPtr type_dispatch_for_aggregate_function_skew(const 
DataTypes& argument_types,
+                                                               const bool 
result_is_nullable,
+                                                               bool 
nullable_input) {
+    using StatFunctionTemplate = StatFuncOneArg<T, 3>;
+
+    if (nullable_input) {
+        return creator_without_type::create_ignore_nullable<
+                AggregateFunctionVarianceSimple<StatFunctionTemplate, true>>(
+                argument_types, result_is_nullable, 
STATISTICS_FUNCTION_KIND::SKEW_POP);
+    } else {
+        return creator_without_type::create_ignore_nullable<
+                AggregateFunctionVarianceSimple<StatFunctionTemplate, false>>(
+                argument_types, result_is_nullable, 
STATISTICS_FUNCTION_KIND::SKEW_POP);
+    }
+};
+
+AggregateFunctionPtr create_aggregate_function_skew(const std::string& name,
+                                                    const DataTypes& 
argument_types,
+                                                    const bool 
result_is_nullable) {
+    if (argument_types.size() != 1) {
+        LOG(WARNING) << "aggregate function " << name << " requires exactly 1 
argument";
+        return nullptr;
+    }
+
+    if (!result_is_nullable) {
+        LOG(WARNING) << "aggregate function " << name << " requires nullable 
result type";
+        return nullptr;
+    }
+
+    const bool nullable_input = argument_types[0]->is_nullable();
+    WhichDataType type(remove_nullable(argument_types[0]));
+
+#define DISPATCH(TYPE)                                                         
                    \
+    if (type.idx == TypeIndex::TYPE)                                           
                    \
+        return type_dispatch_for_aggregate_function_skew<TYPE>(argument_types, 
result_is_nullable, \
+                                                               nullable_input);
+    FOR_NUMERIC_TYPES(DISPATCH)
+#undef DISPATCH
+
+    LOG(WARNING) << "unsupported input type " << argument_types[0]->get_name()
+                 << " for aggregate function " << name;
+    return nullptr;
+}
+
+void register_aggregate_function_skewness(AggregateFunctionSimpleFactory& 
factory) {
+    factory.register_function_both("skew", create_aggregate_function_skew);
+    factory.register_alias("skew", "skew_pop");
+    factory.register_alias("skew", "skewness");
+}
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/aggregate_function_statistic.h 
b/be/src/vec/aggregate_functions/aggregate_function_statistic.h
new file mode 100644
index 00000000000..a1fd4395eb8
--- /dev/null
+++ b/be/src/vec/aggregate_functions/aggregate_function_statistic.h
@@ -0,0 +1,163 @@
+// 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 <cmath>
+#include <cstdint>
+#include <string>
+#include <type_traits>
+
+#include "common/exception.h"
+#include "common/status.h"
+#include "moments.h"
+#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/aggregate_functions/moments.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 {
+
+enum class STATISTICS_FUNCTION_KIND : uint8_t { SKEW_POP, KURT_POP };
+
+inline std::string to_string(STATISTICS_FUNCTION_KIND kind) {
+    switch (kind) {
+    case STATISTICS_FUNCTION_KIND::SKEW_POP:
+        return "skewness";
+    case STATISTICS_FUNCTION_KIND::KURT_POP:
+        return "kurtosis";
+    default:
+        return "Unknown";
+    }
+}
+
+template <typename T, std::size_t _level>
+struct StatFuncOneArg {
+    using Type = T;
+    using Data = VarMoments<Float64, _level>;
+};
+
+template <typename StatFunc, bool NullableInput>
+class AggregateFunctionVarianceSimple
+        : public IAggregateFunctionDataHelper<
+                  typename StatFunc::Data,
+                  AggregateFunctionVarianceSimple<StatFunc, NullableInput>> {
+public:
+    using InputCol = ColumnVector<typename StatFunc::Type>;
+    using ResultCol = ColumnVector<Float64>;
+
+    explicit AggregateFunctionVarianceSimple(STATISTICS_FUNCTION_KIND kind_,
+                                             const DataTypes& argument_types_)
+            : IAggregateFunctionDataHelper<
+                      typename StatFunc::Data,
+                      AggregateFunctionVarianceSimple<StatFunc, 
NullableInput>>(argument_types_),
+              kind(kind_) {
+        DCHECK(!argument_types_.empty());
+    }
+
+    String get_name() const override { return to_string(kind); }
+
+    DataTypePtr get_return_type() const override {
+        return make_nullable(std::make_shared<DataTypeFloat64>());
+    }
+
+    void add(AggregateDataPtr __restrict place, const IColumn** columns, 
ssize_t row_num,
+             Arena*) const override {
+        if constexpr (NullableInput) {
+            const ColumnNullable& column_with_nullable =
+                    assert_cast<const ColumnNullable&, 
TypeCheckOnRelease::DISABLE>(*columns[0]);
+
+            if (column_with_nullable.is_null_at(row_num)) {
+                return;
+            } else {
+                this->data(place).add(assert_cast<const InputCol&, 
TypeCheckOnRelease::DISABLE>(
+                                              
column_with_nullable.get_nested_column())
+                                              .get_data()[row_num]);
+            }
+
+        } else {
+            this->data(place).add(
+                    assert_cast<const InputCol&, 
TypeCheckOnRelease::DISABLE>(*columns[0])
+                            .get_data()[row_num]);
+        }
+    }
+
+    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 {
+        const auto& data = this->data(place);
+        ColumnNullable& dst_column_with_nullable = 
assert_cast<ColumnNullable&>(to);
+        ResultCol* dst_column =
+                
assert_cast<ResultCol*>(&(dst_column_with_nullable.get_nested_column()));
+
+        switch (kind) {
+        case STATISTICS_FUNCTION_KIND::SKEW_POP: {
+            // If input is empty set, we will get NAN from get_population()
+            Float64 var_value = data.get_population();
+            Float64 moments_3 = data.get_moment_3();
+
+            if (std::isnan(var_value) || std::isnan(moments_3) || var_value <= 
0) {
+                dst_column_with_nullable.get_null_map_data().push_back(1);
+                dst_column->insert_default();
+            } else {
+                dst_column_with_nullable.get_null_map_data().push_back(0);
+                dst_column->get_data().push_back(
+                        static_cast<Float64>(moments_3 / pow(var_value, 1.5)));
+            }
+            break;
+        }
+        case STATISTICS_FUNCTION_KIND::KURT_POP: {
+            Float64 var_value = data.get_population();
+            Float64 moments_4 = data.get_moment_4();
+
+            if (std::isnan(var_value) || std::isnan(moments_4) || var_value <= 
0) {
+                dst_column_with_nullable.get_null_map_data().push_back(1);
+                dst_column->insert_default();
+            } else {
+                dst_column_with_nullable.get_null_map_data().push_back(0);
+                // kurtosis = E(X^4) / E(X^2)^2 - 3
+                dst_column->get_data().push_back(
+                        static_cast<Float64>(moments_4 / pow(var_value, 2)) - 
3);
+            }
+            break;
+        }
+        default:
+            throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Unknown 
statistics function kind");
+        }
+    }
+
+private:
+    STATISTICS_FUNCTION_KIND kind;
+};
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/aggregate_functions/moments.h 
b/be/src/vec/aggregate_functions/moments.h
new file mode 100644
index 00000000000..d9db1277470
--- /dev/null
+++ b/be/src/vec/aggregate_functions/moments.h
@@ -0,0 +1,114 @@
+// 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 <stddef.h>
+
+#include "common/exception.h"
+#include "common/status.h"
+#include "vec/io/io_helper.h"
+
+namespace doris::vectorized {
+
+class BufferReadable;
+class BufferWritable;
+
+template <typename T, size_t _level>
+struct VarMoments {
+    // m[1] = sum(x)
+    // m[2] = sum(x^2)
+    // m[3] = sum(x^3)
+    // m[4] = sum(x^4)
+    T m[_level + 1] {};
+
+    void add(T x) {
+        ++m[0];
+        m[1] += x;
+        m[2] += x * x;
+        if constexpr (_level >= 3) m[3] += x * x * x;
+        if constexpr (_level >= 4) m[4] += x * x * x * x;
+    }
+
+    void merge(const VarMoments& rhs) {
+        m[0] += rhs.m[0];
+        m[1] += rhs.m[1];
+        m[2] += rhs.m[2];
+        if constexpr (_level >= 3) m[3] += rhs.m[3];
+        if constexpr (_level >= 4) m[4] += rhs.m[4];
+    }
+
+    void write(BufferWritable& buf) const { write_binary(*this, buf); }
+
+    void read(BufferReadable& buf) { read_binary(*this, buf); }
+
+    T get() const {
+        throw doris::Exception(ErrorCode::INTERNAL_ERROR,
+                               "Variation moments should be obtained by 
'get_population' method");
+    }
+
+    T get_population() const {
+        if (m[0] == 0) return std::numeric_limits<T>::quiet_NaN();
+
+        /// Due to numerical errors, the result can be slightly less than zero,
+        /// but it should be impossible. Trim to zero.
+
+        return std::max(T {}, (m[2] - m[1] * m[1] / m[0]) / m[0]);
+    }
+
+    T get_sample() const {
+        if (m[0] <= 1) return std::numeric_limits<T>::quiet_NaN();
+        return std::max(T {}, (m[2] - m[1] * m[1] / m[0]) / (m[0] - 1));
+    }
+
+    T get_moment_3() const {
+        if constexpr (_level < 3) {
+            throw doris::Exception(
+                    ErrorCode::INTERNAL_ERROR,
+                    "Variation moments should be obtained by 'get_population' 
method");
+        } else {
+            if (m[0] == 0) return std::numeric_limits<T>::quiet_NaN();
+            // to avoid accuracy problem
+            if (m[0] == 1) return 0;
+            /// \[ \frac{1}{m_0} (m_3 - (3 * m_2 - \frac{2 * {m_1}^2}{m_0}) * 
\frac{m_1}{m_0});\]
+            return (m[3] - (3 * m[2] - 2 * m[1] * m[1] / m[0]) * m[1] / m[0]) 
/ m[0];
+        }
+    }
+
+    T get_moment_4() const {
+        if constexpr (_level < 4) {
+            throw doris::Exception(
+                    ErrorCode::INTERNAL_ERROR,
+                    "Variation moments should be obtained by 'get_population' 
method");
+        } else {
+            if (m[0] == 0) return std::numeric_limits<T>::quiet_NaN();
+            // to avoid accuracy problem
+            if (m[0] == 1) return 0;
+            /// \[ \frac{1}{m_0}(m_4 - (4 * m_3 - (6 * m_2 - \frac{3 * 
m_1^2}{m_0} ) \frac{m_1}{m_0})\frac{m_1}{m_0})\]
+            return (m[4] -
+                    (4 * m[3] - (6 * m[2] - 3 * m[1] * m[1] / m[0]) * m[1] / 
m[0]) * m[1] / m[0]) /
+                   m[0];
+        }
+    }
+
+    void reset() {
+        m = {};
+        return;
+    }
+};
+
+} // 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 28b1352eaf4..6889adc6b6c 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
@@ -43,6 +43,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.Histogram;
 import org.apache.doris.nereids.trees.expressions.functions.agg.HllUnion;
 import org.apache.doris.nereids.trees.expressions.functions.agg.HllUnionAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.IntersectCount;
+import org.apache.doris.nereids.trees.expressions.functions.agg.Kurt;
 import org.apache.doris.nereids.trees.expressions.functions.agg.MapAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
 import org.apache.doris.nereids.trees.expressions.functions.agg.MaxBy;
@@ -64,6 +65,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.QuantileUnion;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Retention;
 import org.apache.doris.nereids.trees.expressions.functions.agg.SequenceCount;
 import org.apache.doris.nereids.trees.expressions.functions.agg.SequenceMatch;
+import org.apache.doris.nereids.trees.expressions.functions.agg.Skew;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Stddev;
 import org.apache.doris.nereids.trees.expressions.functions.agg.StddevSamp;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
@@ -145,7 +147,9 @@ public class BuiltinAggregateFunctions implements 
FunctionHelper {
             agg(TopNWeighted.class, "topn_weighted"),
             agg(Variance.class, "var_pop", "variance_pop", "variance"),
             agg(VarianceSamp.class, "var_samp", "variance_samp"),
-            agg(WindowFunnel.class, "window_funnel")
+            agg(WindowFunnel.class, "window_funnel"),
+            agg(Skew.class, "skew", "skew_pop", "skewness"),
+            agg(Kurt.class, "kurt", "kurt_pop", "kurtosis")
     );
 
     public final Set<String> aggFuncNames = aggregateFunctions.stream()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
index 74b2778373e..4c587f8bb95 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionSet.java
@@ -1914,6 +1914,42 @@ public class FunctionSet<T> {
                 Lists.<Type>newArrayList(Type.DOUBLE, Type.DOUBLE), 
Type.DOUBLE, Type.DOUBLE,
                 "", "", "", "", "", "", "",
                 false, false, false, true));
+
+
+        List<String> skewnessAndKurtosis = Lists.newArrayList("skew", 
"skew_pop", "skewness", "kurt",
+                "kurt_pop", "kurtosis");
+        skewnessAndKurtosis.addAll(skewnessAndKurtosis);
+
+        for (String name : skewnessAndKurtosis) {
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.TINYINT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.SMALLINT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.INT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.BIGINT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.LARGEINT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.FLOAT), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+            addBuiltin(AggregateFunction.createBuiltin(name,
+                    Lists.<Type>newArrayList(Type.DOUBLE), Type.DOUBLE, 
Type.DOUBLE,
+                    "", "", "", "", "", "", "",
+                    false, false, false, true));
+        }
     }
 
     public Map<String, List<Function>> getVectorizedFunctions() {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Kurt.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Kurt.java
new file mode 100644
index 00000000000..13b24838e2e
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Kurt.java
@@ -0,0 +1,79 @@
+// 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.AlwaysNullable;
+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.BigIntType;
+import org.apache.doris.nereids.types.DoubleType;
+import org.apache.doris.nereids.types.FloatType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.SmallIntType;
+import org.apache.doris.nereids.types.TinyIntType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * AggregateFunction 'Kurt'.
+ */
+public class Kurt extends AggregateFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
AlwaysNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(BigIntType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(IntegerType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(SmallIntType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(TinyIntType.INSTANCE),
+            FunctionSignature.ret(DoubleType.INSTANCE).args(FloatType.INSTANCE)
+    );
+
+    public Kurt(Expression arg1) {
+        this(false, arg1);
+    }
+
+    public Kurt(boolean distinct, Expression arg1) {
+        super("kurt", distinct, arg1);
+    }
+
+    /**
+     * withDistinctAndChildren.
+     */
+    @Override
+    public Kurt withDistinctAndChildren(boolean distinct, List<Expression> 
children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new Kurt(distinct, children.get(0));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitKurt(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/Skew.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Skew.java
new file mode 100644
index 00000000000..4041b7a3863
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Skew.java
@@ -0,0 +1,80 @@
+// 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.AlwaysNullable;
+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.BigIntType;
+import org.apache.doris.nereids.types.DoubleType;
+import org.apache.doris.nereids.types.FloatType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.SmallIntType;
+import org.apache.doris.nereids.types.TinyIntType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * AggregateFunction 'Skew'.
+ */
+
+public class Skew extends AggregateFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
AlwaysNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(DoubleType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(BigIntType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(IntegerType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(SmallIntType.INSTANCE),
+            
FunctionSignature.ret(DoubleType.INSTANCE).args(TinyIntType.INSTANCE),
+            FunctionSignature.ret(DoubleType.INSTANCE).args(FloatType.INSTANCE)
+    );
+
+    public Skew(Expression arg1) {
+        this(false, arg1);
+    }
+
+    public Skew(boolean distinct, Expression arg1) {
+        super("skew", distinct, arg1);
+    }
+
+    /**
+     * withDistinctAndChildren.
+     */
+    @Override
+    public Skew withDistinctAndChildren(boolean distinct, List<Expression> 
children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new Skew(distinct, children.get(0));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitSkew(this, context);
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+}
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 abe8044c28c..b0f39ca6f7e 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
@@ -44,6 +44,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.Histogram;
 import org.apache.doris.nereids.trees.expressions.functions.agg.HllUnion;
 import org.apache.doris.nereids.trees.expressions.functions.agg.HllUnionAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.IntersectCount;
+import org.apache.doris.nereids.trees.expressions.functions.agg.Kurt;
 import org.apache.doris.nereids.trees.expressions.functions.agg.MapAgg;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
 import org.apache.doris.nereids.trees.expressions.functions.agg.MaxBy;
@@ -66,6 +67,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.QuantileUnion;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Retention;
 import org.apache.doris.nereids.trees.expressions.functions.agg.SequenceCount;
 import org.apache.doris.nereids.trees.expressions.functions.agg.SequenceMatch;
+import org.apache.doris.nereids.trees.expressions.functions.agg.Skew;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Stddev;
 import org.apache.doris.nereids.trees.expressions.functions.agg.StddevSamp;
 import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
@@ -210,6 +212,10 @@ public interface AggregateFunctionVisitor<R, C> {
         return visitAggregateFunction(intersectCount, context);
     }
 
+    default R visitKurt(Kurt kurt, C context) {
+        return visitAggregateFunction(kurt, context);
+    }
+
     default R visitMapAgg(MapAgg mapAgg, C context) {
         return visitAggregateFunction(mapAgg, context);
     }
@@ -278,6 +284,10 @@ public interface AggregateFunctionVisitor<R, C> {
         return visitNullableAggregateFunction(sequenceMatch, context);
     }
 
+    default R visitSkew(Skew skew, C context) {
+        return visitAggregateFunction(skew, context);
+    }
+
     default R visitStddev(Stddev stddev, C context) {
         return visitNullableAggregateFunction(stddev, context);
     }
diff --git 
a/regression-test/data/query_p0/aggregate/aggregate_function_kurt.out 
b/regression-test/data/query_p0/aggregate/aggregate_function_kurt.out
new file mode 100644
index 00000000000..362bd25d078
--- /dev/null
+++ b/regression-test/data/query_p0/aggregate/aggregate_function_kurt.out
@@ -0,0 +1,52 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql_empty_1 --
+\N     \N
+
+-- !sql_empty_2 --
+
+-- !sql_1 --
+\N     \N
+
+-- !sql_2 --
+\N     \N
+
+-- !sql_3 --
+0.16212458373485106    -1.4999999999999993
+
+-- !sql_4 --
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+
+-- !sql_distinct_1 --
+-0.7014368047529627
+
+-- !sql_distinct_2 --
+-1.4999999999999993
+
+-- !sql_distinct_3 --
+\N
+\N
+\N
+\N
+\N
+
+-- !sql_distinct_4 --
+\N
+\N
+\N
+\N
+\N
+
+-- !sql_5 --
+0.16212458373485106    -1.4999999999999993
+
+-- !sql_6 --
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+
diff --git 
a/regression-test/data/query_p0/aggregate/aggregate_function_skew.out 
b/regression-test/data/query_p0/aggregate/aggregate_function_skew.out
new file mode 100644
index 00000000000..3320371dfbb
--- /dev/null
+++ b/regression-test/data/query_p0/aggregate/aggregate_function_skew.out
@@ -0,0 +1,52 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !sql_empty_1 --
+\N     \N
+
+-- !sql_empty_2 --
+
+-- !sql_1 --
+\N     \N
+
+-- !sql_2 --
+\N     \N
+
+-- !sql_3 --
+1.4337199628825619     0.675885787569108
+
+-- !sql_4 --
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+
+-- !sql_distinct_1 --
+1.1135657469022011
+
+-- !sql_distinct_2 --
+0.675885787569108
+
+-- !sql_distinct_3 --
+\N
+\N
+\N
+\N
+\N
+
+-- !sql_distinct_4 --
+\N
+\N
+\N
+\N
+\N
+
+-- !sql_5 --
+1.4337199628825619     0.675885787569108
+
+-- !sql_6 --
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+\N     \N
+
diff --git 
a/regression-test/suites/query_p0/aggregate/aggregate_function_kurt.groovy 
b/regression-test/suites/query_p0/aggregate/aggregate_function_kurt.groovy
new file mode 100644
index 00000000000..0e475467a16
--- /dev/null
+++ b/regression-test/suites/query_p0/aggregate/aggregate_function_kurt.groovy
@@ -0,0 +1,78 @@
+// 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("aggregate_function_kurt") {
+    sql """
+        drop table if exists aggregate_function_kurt;
+    """
+    sql"""
+       create table aggregate_function_kurt (tag int, val1 double not null, 
val2 double null) distributed by hash(tag) buckets 10 
properties('replication_num' = '1');
+    """
+
+    qt_sql_empty_1 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt;
+    """
+    qt_sql_empty_2 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt 
group by tag;
+    """
+
+    sql """
+       insert into aggregate_function_kurt values (1, -10.0, -10.0);
+    """
+
+    qt_sql_1 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt;
+    """
+    qt_sql_2 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt 
group by tag;
+    """
+
+    sql """
+       insert into aggregate_function_kurt values (2, -20.0, NULL), (3, 100, 
NULL), (4, 100, 100), (5,1000, 1000);
+    """
+    qt_sql_3 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt;
+    """
+    qt_sql_4 """
+        select kurtosis(val1),kurtosis(val2) from aggregate_function_kurt 
group by tag;
+    """
+
+    qt_sql_distinct_1 """
+        select kurtosis(distinct val1) from aggregate_function_kurt;
+    """
+    qt_sql_distinct_2 """
+        select kurtosis(distinct val2) from aggregate_function_kurt;
+    """
+
+    qt_sql_distinct_3 """
+        select kurtosis(distinct val1) from aggregate_function_kurt group by 
tag;
+    """
+    qt_sql_distinct_4 """
+        select kurtosis(distinct val2) from aggregate_function_kurt group by 
tag;
+    """
+
+    sql """
+        insert into aggregate_function_kurt select * from 
aggregate_function_kurt;
+    """
+
+    qt_sql_5 """
+        select kurt(val1),kurt_pop(val2) from aggregate_function_kurt;
+    """
+    qt_sql_6 """
+        select kurt(val1),kurt_pop(val2) from aggregate_function_kurt group by 
tag;
+    """
+}
\ No newline at end of file
diff --git 
a/regression-test/suites/query_p0/aggregate/aggregate_function_skew.groovy 
b/regression-test/suites/query_p0/aggregate/aggregate_function_skew.groovy
new file mode 100644
index 00000000000..b36e354cc48
--- /dev/null
+++ b/regression-test/suites/query_p0/aggregate/aggregate_function_skew.groovy
@@ -0,0 +1,78 @@
+// 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("aggregate_function_skew") {
+    sql """
+        drop table if exists aggregate_function_skew;
+    """
+    sql"""
+       create table aggregate_function_skew (tag int, val1 double not null, 
val2 double null) distributed by hash(tag) buckets 10 
properties('replication_num' = '1');
+    """
+
+    qt_sql_empty_1 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew;
+    """
+    qt_sql_empty_2 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew 
group by tag;
+    """
+
+    sql """
+       insert into aggregate_function_skew values (1, -10.0, -10.0);
+    """
+
+    qt_sql_1 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew;
+    """
+    qt_sql_2 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew 
group by tag;
+    """
+
+    sql """
+       insert into aggregate_function_skew values (2, -20.0, NULL), (3, 100, 
NULL), (4, 100, 100), (5,1000, 1000);
+    """
+    qt_sql_3 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew;
+    """
+    qt_sql_4 """
+        select skewness(val1),skewness(val2) from aggregate_function_skew 
group by tag;
+    """
+
+    qt_sql_distinct_1 """
+        select skewness(distinct val1) from aggregate_function_skew;
+    """
+    qt_sql_distinct_2 """
+        select skewness(distinct val2) from aggregate_function_skew;
+    """
+
+    qt_sql_distinct_3 """
+        select skewness(distinct val1) from aggregate_function_skew group by 
tag;
+    """
+    qt_sql_distinct_4 """
+        select skewness(distinct val2) from aggregate_function_skew group by 
tag;
+    """
+
+    sql """
+        insert into aggregate_function_skew select * from 
aggregate_function_skew;
+    """
+
+    qt_sql_5 """
+        select skew(val1),skew_pop(val2) from aggregate_function_skew;
+    """
+    qt_sql_6 """
+        select skew(val1),skew_pop(val2) from aggregate_function_skew group by 
tag;
+    """
+}
\ No newline at end of file


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


Reply via email to