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

zhangstar333 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 d3e3c1bc8ec [feature](function) support format function (#47102)
d3e3c1bc8ec is described below

commit d3e3c1bc8ec7e75721511f54c1ca13ea8586eee8
Author: zhangstar333 <[email protected]>
AuthorDate: Thu Aug 14 11:13:11 2025 +0800

    [feature](function) support format function (#47102)
    
    ### What problem does this PR solve?
    Problem Summary:
    support parse_data_size,format,format_number function
    doc: https://github.com/apache/doris-website/pull/1825
---
 be/src/vec/functions/function_format.cpp           | 246 +++++++++++++++++++++
 be/src/vec/functions/function_string.cpp           |  68 ++++++
 be/src/vec/functions/simple_function_factory.h     |   2 +
 .../doris/catalog/BuiltinScalarFunctions.java      |   6 +
 .../trees/expressions/functions/scalar/Format.java |  80 +++++++
 .../expressions/functions/scalar/FormatNumber.java |  69 ++++++
 .../functions/scalar/ParseDataSize.java            |  70 ++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  15 ++
 gensrc/script/doris_builtins_functions.py          |  10 +
 .../string_functions/test_format_functions.out     | Bin 0 -> 2696 bytes
 .../string_functions/test_format_functions.groovy  | 160 ++++++++++++++
 11 files changed, 726 insertions(+)

diff --git a/be/src/vec/functions/function_format.cpp 
b/be/src/vec/functions/function_format.cpp
new file mode 100644
index 00000000000..5323234e222
--- /dev/null
+++ b/be/src/vec/functions/function_format.cpp
@@ -0,0 +1,246 @@
+// 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 <glog/logging.h>
+
+#include <cstdio>
+#include <regex>
+#include <vector>
+
+#include "common/status.h"
+#include "runtime/define_primitive_type.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_vector.h"
+#include "vec/common/assert_cast.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/functions/cast_type_to_either.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+class FunctionFormatNumber : public IFunction {
+public:
+    static constexpr auto name = "format_number";
+
+    static constexpr const char* UNITS[6] = {"", "K", "M", "B", "T", "Q"};
+
+    static FunctionPtr create() { return 
std::make_shared<FunctionFormatNumber>(); }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    bool is_variadic() const override { return false; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeString>();
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        auto column = block.get_by_position(arguments[0]).column;
+        const auto& column_data = assert_cast<const 
ColumnFloat64*>(column.get())->get_data();
+        auto col_res = ColumnString::create();
+        fmt::memory_buffer buffer;
+
+        for (auto i = 0; i < input_rows_count; ++i) {
+            auto res_data = format_number(buffer, column_data[i]);
+            col_res->insert_data(res_data.data(), res_data.length());
+        }
+        block.replace_by_position(result, std::move(col_res));
+        return Status::OK();
+    }
+
+    std::string format_number(fmt::memory_buffer& buffer, double number) const 
{
+        buffer.clear();
+        double abs_number = std::abs(number);
+        int unit_index = 0;
+        while (abs_number >= 1000 && unit_index < 5) {
+            abs_number /= 1000;
+            ++unit_index;
+        }
+        if (number < 0) {
+            fmt::format_to(buffer, "-");
+        }
+        if (abs_number == 1) {
+            //eg: 1000 ---> 1K
+            fmt::format_to(buffer, "{}", abs_number);
+        } else if (abs_number < 10) {
+            //eg: 1239 ---> 1.24K only want to show 2 decimal
+            fmt::format_to(buffer, "{:.2f}", abs_number);
+        } else if (abs_number < 100) {
+            //eg: 12399999 ---> 12.4M only want to show 1 decimal
+            fmt::format_to(buffer, "{:.1f}", abs_number);
+        } else {
+            // eg: 999999999999999 ---> 1000T only want to show 0 decimal
+            fmt::format_to(buffer, "{:.0f}", abs_number);
+        }
+        fmt::format_to(buffer, UNITS[unit_index]);
+        return fmt::to_string(buffer);
+    }
+};
+
+class FunctionFormat : public IFunction {
+public:
+    static constexpr auto name = "format";
+
+    static FunctionPtr create() { return std::make_shared<FunctionFormat>(); }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 0; }
+
+    bool is_variadic() const override { return true; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeString>();
+    }
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        uint32_t result, size_t input_rows_count) const 
override {
+        DCHECK_GE(arguments.size(), 2);
+        bool valid =
+                cast_type(block.get_by_position(arguments[1]).type.get(), 
[&](const auto& type) {
+                    using DataType = std::decay_t<decltype(type)>;
+                    using ColVecData =
+                            std::conditional_t<is_number(DataType::PType),
+                                               ColumnVector<DataType::PType>, 
ColumnString>;
+                    if (auto col = check_and_get_column<ColVecData>(
+                                           
block.get_by_position(arguments[1]).column.get()) ||
+                                   
is_column_const(*block.get_by_position(arguments[1]).column)) {
+                        execute_inner<ColVecData, DataType::PType>(block, 
arguments, result,
+                                                                   
input_rows_count);
+                        return true;
+                    }
+                    return false;
+                });
+        if (!valid) {
+            return Status::RuntimeError(
+                    "{}'s argument does not match the expected data type, 
type: {}, column: {}",
+                    get_name(), 
block.get_by_position(arguments[1]).type->get_name(),
+                    
block.get_by_position(arguments[1]).column->dump_structure());
+        }
+        return Status::OK();
+    }
+
+    template <typename F>
+    static bool cast_type(const IDataType* type, F&& f) {
+        return cast_type_to_either<DataTypeInt8, DataTypeInt16, DataTypeInt32, 
DataTypeInt64,
+                                   DataTypeInt128, DataTypeFloat32, 
DataTypeFloat64,
+                                   DataTypeString>(type, std::forward<F>(f));
+    }
+
+    template <typename ColVecData, PrimitiveType T>
+    void execute_inner(Block& block, const ColumnNumbers& arguments, uint32_t 
result,
+                       size_t input_rows_count) const {
+        size_t argument_size = arguments.size();
+        std::vector<ColumnPtr> argument_columns(argument_size);
+        auto result_column = ColumnString::create();
+
+        // maybe most user is format(const, column), so only handle this case 
const column
+        if (argument_size == 2) {
+            std::vector<uint8_t> is_consts(argument_size);
+            std::tie(argument_columns[0], is_consts[0]) =
+                    
unpack_if_const(block.get_by_position(arguments[0]).column);
+            std::tie(argument_columns[1], is_consts[1]) =
+                    
unpack_if_const(block.get_by_position(arguments[1]).column);
+            execute_for_two_argument<ColVecData, T>(argument_columns, 
is_consts,
+                                                    
assert_cast<ColumnString*>(result_column.get()),
+                                                    input_rows_count);
+        } else {
+            for (size_t i = 0; i < argument_size; ++i) {
+                argument_columns[i] = block.get_by_position(arguments[i])
+                                              
.column->convert_to_full_column_if_const();
+            }
+            execute_for_others_arg<ColVecData, T>(argument_columns,
+                                                  
assert_cast<ColumnString*>(result_column.get()),
+                                                  argument_size, 
input_rows_count);
+        }
+
+        block.replace_by_position(result, std::move(result_column));
+    }
+
+    template <typename ColVecData, PrimitiveType T>
+    void execute_for_two_argument(std::vector<ColumnPtr>& argument_columns,
+                                  std::vector<uint8_t>& is_consts, 
ColumnString* result_data_column,
+                                  size_t input_rows_count) const {
+        const auto& format_column = assert_cast<const 
ColumnString&>(*argument_columns[0].get());
+        const auto& value_column = assert_cast<const 
ColVecData&>(*argument_columns[1].get());
+        for (int i = 0; i < input_rows_count; ++i) {
+            auto format =
+                    format_column.get_data_at(index_check_const(i, 
is_consts[0])).to_string_view();
+            std::string res;
+            try {
+                if constexpr (is_string_type(T)) {
+                    auto value = value_column.get_data_at(index_check_const(i, 
is_consts[1]));
+                    res = fmt::format(format, value);
+                } else {
+                    auto value = value_column.get_data()[index_check_const(i, 
is_consts[1])];
+                    res = fmt::format(format, value);
+                }
+            } catch (const std::exception& e) {
+                throw doris::Exception(
+                        ErrorCode::INVALID_ARGUMENT,
+                        "Invalid Input argument \"{}\" of function format, 
error: {}", format,
+                        e.what());
+            }
+            result_data_column->insert_data(res.data(), res.length());
+        }
+    }
+
+    template <typename ColVecData, PrimitiveType T>
+    void execute_for_others_arg(std::vector<ColumnPtr>& argument_columns,
+                                ColumnString* result_data_column, size_t 
argument_size,
+                                size_t input_rows_count) const {
+        const auto& format_column = assert_cast<const 
ColumnString&>(*argument_columns[0].get());
+        for (int i = 0; i < input_rows_count; ++i) {
+            auto format = format_column.get_data_at(i).to_string_view();
+            std::string res;
+            fmt::dynamic_format_arg_store<fmt::format_context> args;
+            if constexpr (is_string_type(T)) {
+                for (int col = 1; col < argument_size; ++col) {
+                    const auto& arg_column_data =
+                            assert_cast<const 
ColVecData&>(*argument_columns[col].get());
+                    args.push_back(arg_column_data.get_data_at(i).to_string());
+                }
+            } else {
+                for (int col = 1; col < argument_size; ++col) {
+                    const auto& arg_column_data =
+                            assert_cast<const 
ColVecData&>(*argument_columns[col].get()).get_data();
+                    args.push_back(arg_column_data[i]);
+                }
+            }
+            try {
+                res = fmt::vformat(format, args);
+            } catch (const std::exception& e) {
+                throw doris::Exception(
+                        ErrorCode::INVALID_ARGUMENT,
+                        "Invalid Input argument \"{}\" of function format, 
error: {}", format,
+                        e.what());
+            }
+            result_data_column->insert_data(res.data(), res.length());
+        }
+    }
+};
+
+void register_function_format(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionFormatNumber>();
+    factory.register_function<FunctionFormat>();
+}
+
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_string.cpp 
b/be/src/vec/functions/function_string.cpp
index 5d88784409e..975873d9668 100644
--- a/be/src/vec/functions/function_string.cpp
+++ b/be/src/vec/functions/function_string.cpp
@@ -67,6 +67,72 @@ struct StringASCII {
     }
 };
 
+struct NameParseDataSize {
+    static constexpr auto name = "parse_data_size";
+};
+
+static const std::map<std::string_view, Int128> UNITS = {
+        {"B", static_cast<Int128>(1)},        {"kB", static_cast<Int128>(1) << 
10},
+        {"MB", static_cast<Int128>(1) << 20}, {"GB", static_cast<Int128>(1) << 
30},
+        {"TB", static_cast<Int128>(1) << 40}, {"PB", static_cast<Int128>(1) << 
50},
+        {"EB", static_cast<Int128>(1) << 60}, {"ZB", static_cast<Int128>(1) << 
70},
+        {"YB", static_cast<Int128>(1) << 80}};
+
+struct ParseDataSize {
+    using ReturnType = DataTypeInt128;
+    static constexpr auto PrimitiveTypeImpl = PrimitiveType::TYPE_STRING;
+    using Type = String;
+    using ReturnColumnType = ColumnInt128;
+
+    static Status vector(const ColumnString::Chars& data, const 
ColumnString::Offsets& offsets,
+                         PaddedPODArray<Int128>& res) {
+        auto size = offsets.size();
+        res.resize(size);
+        for (int i = 0; i < size; ++i) {
+            const char* raw_str = reinterpret_cast<const 
char*>(&data[offsets[i - 1]]);
+            int str_size = offsets[i] - offsets[i - 1];
+            res[i] = parse_data_size(std::string_view(raw_str, str_size));
+        }
+        return Status::OK();
+    }
+
+    static Int128 parse_data_size(const std::string_view& dataSize) {
+        int digit_length = 0;
+        for (char c : dataSize) {
+            if (isdigit(c) || c == '.') {
+                digit_length++;
+            } else {
+                break;
+            }
+        }
+
+        if (digit_length == 0) {
+            throw doris::Exception(ErrorCode::INVALID_ARGUMENT,
+                                   "Invalid Input argument \"{}\" of function 
parse_data_size",
+                                   dataSize);
+        }
+        // 123.45MB--->123.45 : MB
+        double value = 0.0;
+        try {
+            value = std::stod(std::string(dataSize.substr(0, digit_length)));
+        } catch (const std::exception& e) {
+            throw doris::Exception(
+                    ErrorCode::INVALID_ARGUMENT,
+                    "Invalid Input argument \"{}\" of function 
parse_data_size, error: {}",
+                    dataSize, e.what());
+        }
+        auto unit = dataSize.substr(digit_length);
+        auto it = UNITS.find(unit);
+        if (it != UNITS.end()) {
+            return static_cast<__int128>(static_cast<long double>(it->second) 
* value);
+        } else {
+            throw doris::Exception(ErrorCode::INVALID_ARGUMENT,
+                                   "Invalid Input argument \"{}\" of function 
parse_data_size",
+                                   dataSize);
+        }
+    }
+};
+
 struct NameQuote {
     static constexpr auto name = "quote";
 };
@@ -1278,6 +1344,7 @@ template <typename LeftDataType, typename RightDataType>
 using StringFindInSetImpl = StringFunctionImpl<LeftDataType, RightDataType, 
FindInSetOp>;
 
 // ready for regist function
+using FunctionStringParseDataSize = FunctionUnaryToType<ParseDataSize, 
NameParseDataSize>;
 using FunctionStringASCII = FunctionUnaryToType<StringASCII, NameStringASCII>;
 using FunctionStringLength = FunctionUnaryToType<StringLengthImpl, 
NameStringLength>;
 using FunctionCrc32 = FunctionUnaryToType<Crc32Impl, NameCrc32>;
@@ -1314,6 +1381,7 @@ using FunctionStringLPad = FunctionStringPad<StringLPad>;
 using FunctionStringRPad = FunctionStringPad<StringRPad>;
 
 void register_function_string(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionStringParseDataSize>();
     factory.register_function<FunctionStringASCII>();
     factory.register_function<FunctionStringLength>();
     factory.register_function<FunctionCrc32>();
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 56ee9eeb933..713032a191d 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -108,6 +108,7 @@ void register_function_match(SimpleFunctionFactory& 
factory);
 void register_function_tokenize(SimpleFunctionFactory& factory);
 void register_function_url(SimpleFunctionFactory& factory);
 void register_function_ip(SimpleFunctionFactory& factory);
+void register_function_format(SimpleFunctionFactory& factory);
 void register_function_multi_match(SimpleFunctionFactory& factory);
 void register_function_split_by_regexp(SimpleFunctionFactory& factory);
 void register_function_assert_true(SimpleFunctionFactory& factory);
@@ -333,6 +334,7 @@ public:
             register_function_split_by_regexp(instance);
             register_function_assert_true(instance);
             register_function_bit_test(instance);
+            register_function_format(instance);
             register_function_compress(instance);
             register_function_dict_get(instance);
             register_function_dict_get_many(instance);
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index 23a8e3bb02c..7b4290584c8 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -203,6 +203,8 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FindInSet;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FirstSignificantSubdomain;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Floor;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Fmod;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Format;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FormatNumber;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FormatRound;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Fpow;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FromBase64;
@@ -351,6 +353,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.NullOrEmpty;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Nullable;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Overlay;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ParseDataSize;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.ParseUrl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Password;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Pi;
@@ -693,6 +696,8 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(FirstSignificantSubdomain.class, 
"first_significant_subdomain"),
             scalar(Floor.class, "floor"),
             scalar(Fmod.class, "fmod"),
+            scalar(Format.class, "format"),
+            scalar(FormatNumber.class, "format_number"),
             scalar(Fpow.class, "fpow"),
             scalar(FromBase64.class, "from_base64"),
             scalar(FromDays.class, "from_days"),
@@ -842,6 +847,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(Overlay.class, "overlay"),
             scalar(ParseUrl.class, "parse_url"),
             scalar(Password.class, "password"),
+            scalar(ParseDataSize.class, "parse_data_size"),
             scalar(Pi.class, "pi"),
             scalar(Pmod.class, "pmod"),
             scalar(Positive.class, "positive"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Format.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Format.java
new file mode 100644
index 00000000000..9db99bc7895
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Format.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.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.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.LargeIntType;
+import org.apache.doris.nereids.types.SmallIntType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.TinyIntType;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/** Format function */
+public class Format extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
TinyIntType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
SmallIntType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
IntegerType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
BigIntType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
LargeIntType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
FloatType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
DoubleType.INSTANCE),
+            
FunctionSignature.ret(StringType.INSTANCE).varArgs(StringType.INSTANCE, 
StringType.INSTANCE));
+
+    /**
+     * constructor with 2 or more arguments.
+     */
+    public Format(Expression arg0, Expression arg1, Expression... varArgs) {
+        super("format", ExpressionUtils.mergeArguments(arg0, arg1, varArgs));
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public Format withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() >= 2);
+        return new Format(children.get(0), children.get(1),
+                children.subList(2, children.size()).toArray(new 
Expression[0]));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitFormat(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/FormatNumber.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/FormatNumber.java
new file mode 100644
index 00000000000..83270bc2a16
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/FormatNumber.java
@@ -0,0 +1,69 @@
+// 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.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DoubleType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'format_number'. This class is generated by GenerateFunction.
+ */
+public class FormatNumber extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(StringType.INSTANCE).args(DoubleType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public FormatNumber(Expression arg) {
+        super("format_number", arg);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public FormatNumber withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new FormatNumber(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitFormatNumber(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ParseDataSize.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ParseDataSize.java
new file mode 100644
index 00000000000..e2974d4fcac
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/ParseDataSize.java
@@ -0,0 +1,70 @@
+// 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.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.LargeIntType;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'parse_data_size'. This class is generated by 
GenerateFunction.
+ */
+public class ParseDataSize extends ScalarFunction
+        implements UnaryExpression, ExplicitlyCastableSignature, 
PropagateNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            
FunctionSignature.ret(LargeIntType.INSTANCE).args(StringType.INSTANCE)
+    );
+
+    /**
+     * constructor with 1 argument.
+     */
+    public ParseDataSize(Expression arg) {
+        super("parse_data_size", arg);
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public ParseDataSize withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1);
+        return new ParseDataSize(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitParseDataSize(this, context);
+    }
+}
+
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 9cb93572965..33ded080fae 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -212,6 +212,8 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FindInSet;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FirstSignificantSubdomain;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Floor;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Fmod;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Format;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.FormatNumber;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FormatRound;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Fpow;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.FromBase64;
@@ -352,6 +354,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.NullIf;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.NullOrEmpty;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Overlay;
+import 
org.apache.doris.nereids.trees.expressions.functions.scalar.ParseDataSize;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.ParseUrl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Password;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Pi;
@@ -1271,6 +1274,14 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(fmod, context);
     }
 
+    default R visitFormat(Format format, C context) {
+        return visitScalarFunction(format, context);
+    }
+
+    default R visitFormatNumber(FormatNumber formatNumber, C context) {
+        return visitScalarFunction(formatNumber, context);
+    }
+
     default R visitFpow(Fpow fpow, C context) {
         return visitScalarFunction(fpow, context);
     }
@@ -1779,6 +1790,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(randomBytes, context);
     }
 
+    default R visitParseDataSize(ParseDataSize parseDataSize, C context) {
+        return visitScalarFunction(parseDataSize, context);
+    }
+
     default R visitPassword(Password password, C context) {
         return visitScalarFunction(password, context);
     }
diff --git a/gensrc/script/doris_builtins_functions.py 
b/gensrc/script/doris_builtins_functions.py
index 07e1bac8d05..705f8f3539f 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -1742,6 +1742,16 @@ visible_functions = {
         [['url_decode'], 'STRING', ['STRING'], ''],
         [['random_bytes'], 'STRING', ['INT'], ''],
 
+        [['format'], 'STRING', ['STRING', 'TINYINT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'SMALLINT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'INT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'BIGINT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'LARGEINT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'FLOAT', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'DOUBLE', '...'], ''],
+        [['format'], 'STRING', ['STRING', 'STRING', '...'], ''],
+        [['format_number'], 'STRING', ['DOUBLE'], ''],
+        [['parse_data_size'], 'LARGEINT', ['STRING'], ''],
         [['overlay'], 'STRING', ['STRING', 'INT', 'INT', 'STRING'], ''],
         [['strcmp'], 'INT', ['STRING', 'STRING'], 'DEPEND_ON_ARGUMENT']
     ],
diff --git 
a/regression-test/data/query_p0/sql_functions/string_functions/test_format_functions.out
 
b/regression-test/data/query_p0/sql_functions/string_functions/test_format_functions.out
new file mode 100644
index 00000000000..32233d76d0e
Binary files /dev/null and 
b/regression-test/data/query_p0/sql_functions/string_functions/test_format_functions.out
 differ
diff --git 
a/regression-test/suites/query_p0/sql_functions/string_functions/test_format_functions.groovy
 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_format_functions.groovy
new file mode 100644
index 00000000000..f015b2f5893
--- /dev/null
+++ 
b/regression-test/suites/query_p0/sql_functions/string_functions/test_format_functions.groovy
@@ -0,0 +1,160 @@
+// 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("test_format_functions") {
+    sql " drop table if exists test_format_functions"
+    sql """
+        create table test_format_functions (
+            id int,
+            s1 string not null,
+            s2 string null,
+            k1 largeint not null,
+            k2 largeint null
+        )
+        DISTRIBUTED BY HASH(id)
+        PROPERTIES
+        (
+            "replication_num" = "1"
+        );
+    """
+
+    //empty table
+    order_qt_empty_nullable1 "select parse_data_size(s2) from 
test_format_functions"
+    order_qt_empty_nullable2 "select format(s2,'') from test_format_functions"
+    order_qt_empty_nullable3 "select format_number(k2) from 
test_format_functions"
+    order_qt_empty_not_nullable1 "select parse_data_size(s1) from 
test_format_functions"
+    order_qt_empty_not_nullable2 "select format(s1,'') from 
test_format_functions"
+    order_qt_empty_not_nullable3 "select format_number(k1) from 
test_format_functions"
+
+    //null / const
+    order_qt_empty_null1 "select parse_data_size(NULL)"
+    order_qt_empty_null2 "select format_number(NULL)"
+    order_qt_empty_null3 "select format(NULL,'')"
+    order_qt_empty_null4 "select format('',NULL)"
+    order_qt_empty_null5 "select format(NULL,NULL)"
+    
+    //valid data
+    order_qt_empty_const1 "select parse_data_size('0B')"
+    order_qt_empty_const2 "select parse_data_size('1B')"
+    order_qt_empty_const3 "select parse_data_size('1.2B')"
+    order_qt_empty_const4 "select parse_data_size('1.9B')"
+    order_qt_empty_const5 "select parse_data_size('2.2kB')"
+    order_qt_empty_const6 "select parse_data_size('2.23kB')"
+    order_qt_empty_const7 "select parse_data_size('2.234kB')"
+    order_qt_empty_const8 "select parse_data_size('3MB')"
+    order_qt_empty_const9 "select parse_data_size('4GB')"
+    order_qt_empty_const10 "select parse_data_size('4TB')"
+    order_qt_empty_const11 "select parse_data_size('5PB')"
+    order_qt_empty_const12 "select parse_data_size('6EB')"
+    order_qt_empty_const13 "select parse_data_size('7ZB')"
+    order_qt_empty_const14 "select parse_data_size('8YB')"
+    order_qt_empty_const15 "select parse_data_size('6917529027641081856EB')"
+    order_qt_empty_const16 "select parse_data_size('69175290276410818560EB')"
+    //invalid data    
+    test {
+         sql """ select parse_data_size(''); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('0'); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('10KB'); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('KB'); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('-1B'); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('12345K'); """
+         exception "Invalid Input argument"
+    }
+    test {
+         sql """ select parse_data_size('A12345B'); """
+         exception "Invalid Input argument"
+    }
+
+    
+    //format_number
+    order_qt_format_number_1 "select format_number(123);"
+    order_qt_format_number_2 "select format_number(12345);"
+    order_qt_format_number_3 "select format_number(12399);"
+    order_qt_format_number_4 "select format_number(12345678);"
+    order_qt_format_number_5 "select format_number(12399999);"
+    order_qt_format_number_6 "select format_number(12345678901);"
+    order_qt_format_number_7 "select format_number(12399999999);"
+    order_qt_format_number_8 "select format_number(1234.5);"
+    order_qt_format_number_9 "select format_number(1239.9);"
+    order_qt_format_number_10 "select format_number(1234567.8);"
+    order_qt_format_number_11 "select format_number(1239999.9);"
+    order_qt_format_number_12 "select format_number(1234567890.1);"
+    order_qt_format_number_13 "select format_number(1239999999.9);"
+    order_qt_format_number_14 "select format_number(-999);"
+    order_qt_format_number_15 "select format_number(-1000);"
+    order_qt_format_number_16 "select format_number(-999999);"
+    order_qt_format_number_17 "select format_number(-1000000);"
+    order_qt_format_number_18 "select format_number(-999999999);"
+    order_qt_format_number_19 "select format_number(-1000000000);"
+    order_qt_format_number_20 "select format_number(-999999999999);"
+    order_qt_format_number_21 "select format_number(-1000000000000);"
+    order_qt_format_number_22 "select format_number(-999999999999999);"
+    order_qt_format_number_23 "select format_number(-1000000000000000);"
+    order_qt_format_number_24 "select format_number(-9223372036854775808);"
+    order_qt_format_number_25 "select format_number(0);"
+    order_qt_format_number_26 "select format_number(999999);"
+    order_qt_format_number_27 "select format_number(1000000);"
+
+    //format
+    order_qt_format_1 "select format('{}', 123);"
+    order_qt_format_2 "select format('{} of {}', 123, 456);"
+    order_qt_format_3 "select format('{0}{1}', pi(),123);"
+    order_qt_format_4 "select format('{:05}', 8);"
+    order_qt_format_5 "select format('{1}{0}', 'hello', 'world');"
+    order_qt_format_6 "select format('{:.3}', pi());"
+    order_qt_format_7 "select format('{:e}', e());"
+    test {
+         sql """ select format('{asdasdsa}',"asd"); """
+         exception "Invalid Input argument"
+    }
+
+    sql """ insert into test_format_functions values (1, '2.2kB', 
'2.2kB',12345678,12345678); """
+    sql """ insert into test_format_functions values (2, '8YB', 
'8YB',1234567890.1,1234567890.1); """
+    sql """ insert into test_format_functions values (3, '4TB', 
'4TB',-1000000000000000,-1000000000000000); """
+    sql """ insert into test_format_functions values (4, '2.234kB', '2.234kB', 
1234.5, 1234.5); """
+    sql """ insert into test_format_functions values (5, 
'6917529027641081856EB', '6917529027641081856EB',123,123); """
+    sql """ insert into test_format_functions values (6, '0B', 
'0B',999999,999999); """
+    sql """ insert into test_format_functions values (7, "1B", "1B",0,NULL); 
"""
+
+    order_qt_nullable1 "select id,s2,parse_data_size(s2) from 
test_format_functions order by id"
+    order_qt_nullable2 "select id,k2,format_number(k2) from 
test_format_functions order by id"
+
+    order_qt_not_nullable1 "select id,s1,parse_data_size(s1) from 
test_format_functions order by id"
+    order_qt_not_nullable2 "select id,k1,format_number(k1) from 
test_format_functions order by id"
+
+    sql """ insert into test_format_functions values (8, '{:>6}', 
'{:>6}',1234567890.1234,1234567890.1234); """
+    sql """ insert into test_format_functions values (9, '{}', 
'{}',-1000000000000000,-1000000000000000); """
+    sql """ insert into test_format_functions values (10, '{:06}', '{:06}', 
pi(), pi()); """
+    order_qt_nullable3 "select id,s2,format(s2,k1) from test_format_functions 
where id in (7,8,9,10) order by id"
+    order_qt_not_nullable3 "select id,s1,format(s1,1234.4) from 
test_format_functions where id in (7,8,9,10) order by id"
+
+}


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


Reply via email to