You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zy...@apache.org on 2023/11/17 02:27:16 UTC

(doris) branch master updated: [feature](function) support ip function ipv4stringtonum(ordefault, ornull), inet_aton (#25510)

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

zykkk 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 334260dff78 [feature](function) support ip function ipv4stringtonum(ordefault, ornull), inet_aton (#25510)
334260dff78 is described below

commit 334260dff782be7fa5a8a3bf4ea702b95633f0ad
Author: Chester <42...@users.noreply.github.com>
AuthorDate: Fri Nov 17 10:27:07 2023 +0800

    [feature](function) support ip function ipv4stringtonum(ordefault, ornull), inet_aton (#25510)
---
 be/src/vec/common/format_ip.h                      |  95 +++++++++++++++
 be/src/vec/functions/function_ip.cpp               |   5 +
 be/src/vec/functions/function_ip.h                 | 133 +++++++++++++++++++++
 .../sql-functions/ip-functions/inet-aton.md        |  61 ++++++++++
 .../ip-functions/ipv4-string-to-num-or-default.md  |  69 +++++++++++
 .../ip-functions/ipv4-string-to-num-or-null.md     |  69 +++++++++++
 .../ip-functions/ipv4-string-to-num.md             |  61 ++++++++++
 docs/sidebars.json                                 |   6 +-
 .../sql-functions/ip-functions/inet-aton.md        |  61 ++++++++++
 .../ip-functions/ipv4-string-to-num-or-default.md  |  69 +++++++++++
 .../ip-functions/ipv4-string-to-num-or-null.md     |  69 +++++++++++
 .../ip-functions/ipv4-string-to-num.md             |  61 ++++++++++
 .../doris/catalog/BuiltinScalarFunctions.java      |   6 +
 .../functions/scalar/Ipv4StringToNum.java          |  67 +++++++++++
 .../functions/scalar/Ipv4StringToNumOrDefault.java |  67 +++++++++++
 .../functions/scalar/Ipv4StringToNumOrNull.java    |  67 +++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  15 +++
 gensrc/script/doris_builtins_functions.py          |   6 +
 .../data/nereids_function_p0/ip_functions.out      |  11 ++
 .../ip_functions/test_ip_functions.out             |  12 ++
 .../suites/nereids_function_p0/ip_functions.groovy |   4 +
 .../ip_functions/test_ip_functions.groovy          |   4 +
 22 files changed, 1017 insertions(+), 1 deletion(-)

diff --git a/be/src/vec/common/format_ip.h b/be/src/vec/common/format_ip.h
index ce61362d286..ff440b11e21 100644
--- a/be/src/vec/common/format_ip.h
+++ b/be/src/vec/common/format_ip.h
@@ -20,6 +20,8 @@
 
 #pragma once
 
+#include <vec/common/string_utils/string_utils.h>
+
 #include <algorithm>
 #include <array>
 #include <bit>
@@ -32,6 +34,9 @@ constexpr size_t IPV4_MAX_TEXT_LENGTH = 15; /// Does not count tail zero byte.
 constexpr size_t IPV6_MAX_TEXT_LENGTH = 39;
 constexpr size_t IPV4_MIN_NUM_VALUE = 0;          //num value of '0.0.0.0'
 constexpr size_t IPV4_MAX_NUM_VALUE = 4294967295; //num value of '255.255.255.255'
+constexpr int IPV4_MAX_OCTET_VALUE = 255;         //max vulue of octet
+constexpr size_t IPV4_OCTET_BITS = 8;
+constexpr size_t DECIMAL_BASE = 10;
 
 namespace doris::vectorized {
 
@@ -96,4 +101,94 @@ inline void formatIPv4(const unsigned char* src, char*& dst, uint8_t mask_tail_o
     formatIPv4(src, 4, dst, mask_tail_octets, mask_string);
 }
 
+/** Unsafe (no bounds-checking for src nor dst), optimized version of parsing IPv4 string.
+ *
+ * Parses the input string `src` and stores binary host-endian value into buffer pointed by `dst`,
+ * which should be long enough.
+ * That is "127.0.0.1" becomes 0x7f000001.
+ *
+ * In case of failure doesn't modify buffer pointed by `dst`.
+ *
+ * WARNING - this function is adapted to work with ReadBuffer, where src is the position reference (ReadBuffer::position())
+ *           and eof is the ReadBuffer::eof() - therefore algorithm below does not rely on buffer's continuity.
+ *           To parse strings use overloads below.
+ *
+ * @param src         - iterator (reference to pointer) over input string - warning - continuity is not guaranteed.
+ * @param eof         - function returning true if iterator riched the end - warning - can break iterator's continuity.
+ * @param dst         - where to put output bytes, expected to be non-null and at IPV4_BINARY_LENGTH-long.
+ * @param first_octet - preparsed first octet
+ * @return            - true if parsed successfully, false otherwise.
+ */
+template <typename T, typename EOFfunction>
+    requires(std::is_same<typename std::remove_cv<T>::type, char>::value)
+inline bool parseIPv4(T*& src, EOFfunction eof, unsigned char* dst, int64_t first_octet = -1) {
+    if (src == nullptr || first_octet > IPV4_MAX_OCTET_VALUE) {
+        return false;
+    }
+
+    int64_t result = 0;
+    int offset = (IPV4_BINARY_LENGTH - 1) * IPV4_OCTET_BITS;
+    if (first_octet >= 0) {
+        result |= first_octet << offset;
+        offset -= IPV4_OCTET_BITS;
+    }
+
+    for (; true; offset -= IPV4_OCTET_BITS, ++src) {
+        if (eof()) {
+            return false;
+        }
+
+        int64_t value = 0;
+        size_t len = 0;
+        while (is_numeric_ascii(*src) && len <= 3) {
+            value = value * DECIMAL_BASE + (*src - '0');
+            ++len;
+            ++src;
+            if (eof()) {
+                break;
+            }
+        }
+        if (len == 0 || value > IPV4_MAX_OCTET_VALUE || (offset > 0 && (eof() || *src != '.'))) {
+            return false;
+        }
+        result |= value << offset;
+
+        if (offset == 0) {
+            break;
+        }
+    }
+
+    memcpy(dst, &result, sizeof(result));
+    return true;
+}
+
+/// returns pointer to the right after parsed sequence or null on failed parsing
+inline const char* parseIPv4(const char* src, const char* end, unsigned char* dst) {
+    if (parseIPv4(
+                src, [&src, end]() { return src == end; }, dst)) {
+        return src;
+    }
+    return nullptr;
+}
+
+/// returns true if whole buffer was parsed successfully
+inline bool parseIPv4whole(const char* src, const char* end, unsigned char* dst) {
+    return parseIPv4(src, end, dst) == end;
+}
+
+/// returns pointer to the right after parsed sequence or null on failed parsing
+inline const char* parseIPv4(const char* src, unsigned char* dst) {
+    if (parseIPv4(
+                src, []() { return false; }, dst)) {
+        return src;
+    }
+    return nullptr;
+}
+
+/// returns true if whole null-terminated string was parsed successfully
+inline bool parseIPv4whole(const char* src, unsigned char* dst) {
+    const char* end = parseIPv4(src, dst);
+    return end != nullptr && *end == '\0';
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_ip.cpp b/be/src/vec/functions/function_ip.cpp
index 4acd586c96c..762134780db 100644
--- a/be/src/vec/functions/function_ip.cpp
+++ b/be/src/vec/functions/function_ip.cpp
@@ -25,5 +25,10 @@ struct NameFunctionIPv4NumToString {
 void register_function_ip(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionIPv4NumToString<0, NameFunctionIPv4NumToString>>();
     factory.register_alias(NameFunctionIPv4NumToString::name, "inet_ntoa");
+    factory.register_function<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>>();
+    factory.register_function<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Default>>();
+    factory.register_function<FunctionIPv4StringToNum<IPStringToNumExceptionMode::Null>>();
+    factory.register_alias(FunctionIPv4StringToNum<IPStringToNumExceptionMode::Throw>::name,
+                           "inet_aton");
 }
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h
index 4dbbaf47eb6..18ffc655eb8 100644
--- a/be/src/vec/functions/function_ip.h
+++ b/be/src/vec/functions/function_ip.h
@@ -26,6 +26,7 @@
 #include "vec/columns/column_vector.h"
 #include "vec/common/format_ip.h"
 #include "vec/core/column_with_type_and_name.h"
+#include "vec/data_types/data_type_number.h"
 #include "vec/data_types/data_type_string.h"
 #include "vec/functions/function.h"
 #include "vec/functions/simple_function_factory.h"
@@ -117,4 +118,136 @@ public:
                 argument.name, get_name());
     }
 };
+
+enum class IPStringToNumExceptionMode : uint8_t { Throw, Default, Null };
+
+static inline bool tryParseIPv4(const char* pos, Int64& result_value) {
+    return parseIPv4whole(pos, reinterpret_cast<unsigned char*>(&result_value));
+}
+
+template <IPStringToNumExceptionMode exception_mode, typename ToColumn>
+ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray<UInt8>* null_map = nullptr) {
+    const ColumnString* column_string = check_and_get_column<ColumnString>(column.get());
+
+    if (!column_string) {
+        throw Exception(ErrorCode::INVALID_ARGUMENT,
+                        "Illegal column {} of argument of function {}, expected String",
+                        column->get_name());
+    }
+
+    size_t column_size = column_string->size();
+
+    ColumnUInt8::MutablePtr col_null_map_to;
+    ColumnUInt8::Container* vec_null_map_to = nullptr;
+
+    if constexpr (exception_mode == IPStringToNumExceptionMode::Null) {
+        col_null_map_to = ColumnUInt8::create(column_size, false);
+        vec_null_map_to = &col_null_map_to->get_data();
+    }
+
+    auto col_res = ToColumn::create();
+
+    auto& vec_res = col_res->get_data();
+    vec_res.resize(column_size);
+
+    const ColumnString::Chars& vec_src = column_string->get_chars();
+    const ColumnString::Offsets& offsets_src = column_string->get_offsets();
+    size_t prev_offset = 0;
+
+    for (size_t i = 0; i < vec_res.size(); ++i) {
+        if (null_map && (*null_map)[i]) {
+            vec_res[i] = 0;
+            prev_offset = offsets_src[i];
+            if constexpr (exception_mode == IPStringToNumExceptionMode::Null) {
+                (*vec_null_map_to)[i] = true;
+            }
+            continue;
+        }
+
+        const char* src_start = reinterpret_cast<const char*>(&vec_src[prev_offset]);
+        size_t src_length = (i < vec_res.size() - 1) ? (offsets_src[i] - prev_offset)
+                                                     : (vec_src.size() - prev_offset);
+        std::string src(src_start, src_length);
+        bool parse_result = tryParseIPv4(src.c_str(), vec_res[i]);
+
+        if (!parse_result) {
+            if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) {
+                throw Exception(ErrorCode::INVALID_ARGUMENT, "Invalid IPv4 value");
+            } else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) {
+                vec_res[i] = 0;
+            } else if constexpr (exception_mode == IPStringToNumExceptionMode::Null) {
+                (*vec_null_map_to)[i] = true;
+                vec_res[i] = 0;
+            }
+        }
+
+        prev_offset = offsets_src[i];
+    }
+
+    if constexpr (exception_mode == IPStringToNumExceptionMode::Null)
+        return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to));
+
+    return col_res;
+}
+
+template <IPStringToNumExceptionMode exception_mode>
+class FunctionIPv4StringToNum : public IFunction {
+public:
+    static constexpr auto name = exception_mode == IPStringToNumExceptionMode::Throw
+                                         ? "ipv4stringtonum"
+                                         : (exception_mode == IPStringToNumExceptionMode::Default
+                                                    ? "ipv4stringtonumordefault"
+                                                    : "ipv4stringtonumornull");
+
+    static FunctionPtr create() {
+        return std::make_shared<FunctionIPv4StringToNum<exception_mode>>();
+    }
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        if (!is_string(remove_nullable(arguments[0]))) {
+            throw Exception(ErrorCode::INVALID_ARGUMENT,
+                            "Illegal type {} of argument of function {}", arguments[0]->get_name(),
+                            get_name());
+        }
+        auto result_type = std::make_shared<DataTypeInt64>();
+
+        if constexpr (exception_mode == IPStringToNumExceptionMode::Null) {
+            return make_nullable(result_type);
+        }
+
+        return arguments[0]->is_nullable() ? make_nullable(result_type) : result_type;
+    }
+
+    bool use_default_implementation_for_nulls() const override { return false; }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) const override {
+        ColumnPtr column = block.get_by_position(arguments[0]).column;
+        ColumnPtr null_map_column;
+        const NullMap* null_map = nullptr;
+        if (column->is_nullable()) {
+            const auto* column_nullable = assert_cast<const ColumnNullable*>(column.get());
+            column = column_nullable->get_nested_column_ptr();
+            null_map_column = column_nullable->get_null_map_column_ptr();
+            null_map = &column_nullable->get_null_map_data();
+        }
+
+        auto col_res = convertToIPv4<exception_mode, ColumnInt64>(column, null_map);
+
+        if (null_map && !col_res->is_nullable()) {
+            block.replace_by_position(result,
+                                      ColumnNullable::create(IColumn::mutate(col_res),
+                                                             IColumn::mutate(null_map_column)));
+            return Status::OK();
+        }
+
+        block.replace_by_position(result, col_res);
+        return Status::OK();
+    }
+};
+
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md
new file mode 100644
index 00000000000..1120a4852f4
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/inet-aton.md
@@ -0,0 +1,61 @@
+---
+{
+"title": "INET_ATON",
+"language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+## INET_ATON
+
+<version since="dev">
+
+inet_aton
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT INET_ATON(VARCHAR ipv4_string)`
+
+Takes a string containing an IPv4 address in the format A.B.C.D (dot-separated numbers in decimal form). Returns a BIGINT number representing the corresponding IPv4 address in big endian.
+
+### notice
+
+`will return an error if the input string is not a valid IPv4 address`
+
+### example
+```
+mysql> select inet_aton('192.168.0.1'); 
++--------------------------------+ 
+| inet_aton('192.168.0.1') | 
++--------------------------------+ 
+| 3232235521                     | 
++--------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> SELECT inet_aton('192.168');
+ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value
+```
+
+### keywords
+
+INET_ATON, IP
\ No newline at end of file
diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md
new file mode 100644
index 00000000000..54f3c3954da
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md
@@ -0,0 +1,69 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM_OR_DEFAULT",
+"language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNumOrDefault
+
+<version since="dev">
+
+IPv4StringToNumOrDefault
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNumOrDefault(VARCHAR ipv4_string)`
+
+Takes a string containing an IPv4 address in the format A.B.C.D (dot-separated numbers in decimal form). Returns a BIGINT number representing the corresponding IPv4 address in big endian.
+
+### notice
+
+`will return 0 if the input parameter is invalid ipv4 value`
+
+### example
+```
+mysql> select ipv4stringtonumordefault('192.168.0.1'); 
++-----------------------------------------+ 
+| ipv4stringtonumordefault('192.168.0.1') | 
++-----------------------------------------+ 
+| 3232235521                              | 
++-----------------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> select str, ipv4stringtonumordefault(str) from ipv4_str; 
++-----------------+-------------------------------+ 
+|str              | ipv4stringtonumordefault(str) | 
++-----------------+-------------------------------+ 
+| 0.0.0.0         | 0                             | 
+| 127.0.0.1       | 2130706433                    | 
+| 255.255.255.255 | 4294967295                    | 
+| invalid         | 0                             | 
++-----------------+-------------------------------+ 
+4 rows in set (0.01 sec)
+```
+
+### keywords
+
+IPV4STRINGTONUMORDEFAULT, IP
\ No newline at end of file
diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md
new file mode 100644
index 00000000000..372e510d439
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md
@@ -0,0 +1,69 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM_OR_NULL",
+"language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNumOrNull
+
+<version since="dev">
+
+IPv4StringToNumOrNull
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNumOrNull(VARCHAR ipv4_string)`
+
+Takes a string containing an IPv4 address in the format A.B.C.D (dot-separated numbers in decimal form). Returns a BIGINT number representing the corresponding IPv4 address in big endian.
+
+### notice
+
+`will return null if the input parameter is invalid ipv4 value`
+
+### example
+```
+mysql> select ipv4stringtonumornull('192.168.0.1'); 
++--------------------------------------+ 
+| ipv4stringtonumornull('192.168.0.1') | 
++--------------------------------------+ 
+| 3232235521                           | 
++--------------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> select str, ipv4stringtonumornull(str) from ipv4_str; 
++-----------------+----------------------------+ 
+|str              | ipv4stringtonumornull(str) | 
++-----------------+----------------------------+ 
+| 0.0.0.0         | 0                          | 
+| 127.0.0.1       | 2130706433                 | 
+| 255.255.255.255 | 4294967295                 | 
+| invalid         | NULL                       | 
++-----------------+----------------------------+ 
+4 rows in set (0.01 sec)
+```
+
+### keywords
+
+IPV4STRINGTONUMORNULL, IP
\ No newline at end of file
diff --git a/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md
new file mode 100644
index 00000000000..c6c42be9480
--- /dev/null
+++ b/docs/en/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md
@@ -0,0 +1,61 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM",
+"language": "en"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNum
+
+<version since="dev">
+
+IPv4StringToNum
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNum(VARCHAR ipv4_string)`
+
+Takes a string containing an IPv4 address in the format A.B.C.D (dot-separated numbers in decimal form). Returns a BIGINT number representing the corresponding IPv4 address in big endian.
+
+### notice
+
+`will return an error if the input string is not a valid IPv4 address`
+
+### example
+```
+mysql> select ipv4stringtonum('192.168.0.1'); 
++--------------------------------+ 
+| ipv4stringtonum('192.168.0.1') | 
++--------------------------------+ 
+| 3232235521                     | 
++--------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> select ipv4stringtonum('invalid'); 
+ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value
+```
+
+### keywords
+
+IPV4STRINGTONUM, IP
\ No newline at end of file
diff --git a/docs/sidebars.json b/docs/sidebars.json
index 970a2e118ad..0c3855ec594 100644
--- a/docs/sidebars.json
+++ b/docs/sidebars.json
@@ -781,7 +781,11 @@
                             "label": "IP Functions",
                             "items": [
                                 "sql-manual/sql-functions/ip-functions/ipv4-num-to-string",
-                                "sql-manual/sql-functions/ip-functions/inet-ntoa"
+                                "sql-manual/sql-functions/ip-functions/inet-ntoa",
+                                "sql-manual/sql-functions/ip-functions/ipv4-string-to-num",
+                                "sql-manual/sql-functions/ip-functions/inet-aton",
+                                "sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default",
+                                "sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null"
                             ]
                         },
                         {
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md
new file mode 100644
index 00000000000..2260dfbc49d
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/inet-aton.md
@@ -0,0 +1,61 @@
+---
+{
+"title": "INET_ATON",
+"language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+## INET_ATON
+
+<version since="dev">
+
+inet_aton
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT INET_ATON(VARCHAR ipv4_string)`
+
+获取包含 IPv4 地址的字符串,格式为 A.B.C.D(点分隔的十进制数字)。返回一个 BIGINT 数字,表示相应的大端 IPv4 地址。
+
+### notice
+
+`如果输入字符串不是有效的 IPv4 地址,将返回错误`
+
+### example
+```
+mysql> select inet_aton('192.168.0.1'); 
++--------------------------------+ 
+| inet_aton('192.168.0.1') | 
++--------------------------------+ 
+| 3232235521                     | 
++--------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> SELECT inet_aton('192.168');
+ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value
+```
+
+### keywords
+
+INET_ATON, IP
\ No newline at end of file
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md
new file mode 100644
index 00000000000..9132535f50c
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-default.md
@@ -0,0 +1,69 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM_OR_DEFAULT",
+"language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNumOrDefault
+
+<version since="dev">
+
+IPv4StringToNumOrDefault
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNumOrDefault(VARCHAR ipv4_string)`
+
+获取包含 IPv4 地址的字符串,格式为 A.B.C.D(点分隔的十进制数字)。返回一个 BIGINT 数字,表示相应的大端 IPv4 地址。
+
+### notice
+
+`如果输入字符串不是有效的 IPv4 地址,将返回0`
+
+### example
+```
+mysql> select ipv4stringtonumordefault('192.168.0.1'); 
++-----------------------------------------+ 
+| ipv4stringtonumordefault('192.168.0.1') | 
++-----------------------------------------+ 
+| 3232235521                              | 
++-----------------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> select str, ipv4stringtonumordefault(str) from ipv4_str; 
++-----------------+-------------------------------+ 
+|str              | ipv4stringtonumordefault(str) | 
++-----------------+-------------------------------+ 
+| 0.0.0.0         | 0                             | 
+| 127.0.0.1       | 2130706433                    | 
+| 255.255.255.255 | 4294967295                    | 
+| invalid         | 0                             | 
++-----------------+-------------------------------+ 
+4 rows in set (0.01 sec)
+```
+
+### keywords
+
+IPV4STRINGTONUMORDEFAULT, IP
\ No newline at end of file
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md
new file mode 100644
index 00000000000..6d320c8fdbd
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num-or-null.md
@@ -0,0 +1,69 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM_OR_NULL",
+"language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNumOrNull
+
+<version since="dev">
+
+IPv4StringToNumOrNull
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNumOrNull(VARCHAR ipv4_string)`
+
+获取包含 IPv4 地址的字符串,格式为 A.B.C.D(点分隔的十进制数字)。返回一个 BIGINT 数字,表示相应的大端 IPv4 地址。
+
+### notice
+
+`如果输入字符串不是有效的 IPv4 地址,将返回NULL`
+
+### example
+```
+mysql> select ipv4stringtonumornull('192.168.0.1'); 
++--------------------------------------+ 
+| ipv4stringtonumornull('192.168.0.1') | 
++--------------------------------------+ 
+| 3232235521                           | 
++--------------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> select str, ipv4stringtonumornull(str) from ipv4_str; 
++-----------------+----------------------------+ 
+|str              | ipv4stringtonumornull(str) | 
++-----------------+----------------------------+ 
+| 0.0.0.0         | 0                          | 
+| 127.0.0.1       | 2130706433                 | 
+| 255.255.255.255 | 4294967295                 | 
+| invalid         | NULL                       | 
++-----------------+----------------------------+ 
+4 rows in set (0.01 sec)
+```
+
+### keywords
+
+IPV4STRINGTONUMORNULL, IP
\ No newline at end of file
diff --git a/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md
new file mode 100644
index 00000000000..e97087bf010
--- /dev/null
+++ b/docs/zh-CN/docs/sql-manual/sql-functions/ip-functions/ipv4-string-to-num.md
@@ -0,0 +1,61 @@
+---
+{
+"title": "IPV4_STRING_TO_NUM",
+"language": "zh-CN"
+}
+---
+
+<!-- 
+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.
+-->
+
+## IPv4StringToNum
+
+<version since="dev">
+
+IPv4StringToNum
+
+</version>
+
+### description
+
+#### Syntax
+
+`BIGINT IPv4StringToNum(VARCHAR ipv4_string)`
+
+获取包含 IPv4 地址的字符串,格式为 A.B.C.D(点分隔的十进制数字)。返回一个 BIGINT 数字,表示相应的大端 IPv4 地址。
+
+### notice
+
+`如果输入字符串不是有效的 IPv4 地址,将返回错误`
+
+### example
+```
+mysql> select ipv4stringtonum('192.168.0.1'); 
++--------------------------------+ 
+| ipv4stringtonum('192.168.0.1') | 
++--------------------------------+ 
+| 3232235521                     | 
++--------------------------------+ 
+1 row in set (0.01 sec)
+
+mysql> SELECT ipv4stringtonum('192.168');
+ERROR 1105 (HY000): errCode = 2, detailMessage = (172.17.0.2)[CANCELLED][INVALID_ARGUMENT][E33] Invalid IPv4 value
+```
+
+### keywords
+
+IPV4STRINGTONUM, IP
\ No newline at end of file
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 e91f7949f79..5df7b67d36b 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
@@ -190,6 +190,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNum;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrDefault;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrNull;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract;
@@ -582,6 +585,9 @@ public class BuiltinScalarFunctions implements FunctionHelper {
             scalar(InnerProduct.class, "inner_product"),
             scalar(Instr.class, "instr"),
             scalar(Ipv4NumToString.class, "ipv4numtostring", "inet_ntoa"),
+            scalar(Ipv4StringToNum.class, "ipv4stringtonum", "inet_aton"),
+            scalar(Ipv4StringToNumOrDefault.class, "ipv4stringtonumordefault"),
+            scalar(Ipv4StringToNumOrNull.class, "ipv4stringtonumornull"),
             scalar(JsonArray.class, "json_array"),
             scalar(JsonObject.class, "json_object"),
             scalar(JsonQuote.class, "json_quote"),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNum.java
new file mode 100644
index 00000000000..9540fea3073
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNum.java
@@ -0,0 +1,67 @@
+// 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.AlwaysNotNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * scalar function Ipv4StringToNum
+ */
+public class Ipv4StringToNum extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(StringType.INSTANCE));
+
+    public Ipv4StringToNum(Expression arg0) {
+        super("ipv4stringtonum", arg0);
+    }
+
+    @Override
+    public Ipv4StringToNum withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1,
+                "ipv4stringtonum accept 1 args, but got %s (%s)",
+                children.size(),
+                children);
+        return new Ipv4StringToNum(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitIpv4StringToNum(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrDefault.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrDefault.java
new file mode 100644
index 00000000000..4f1f7988e84
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrDefault.java
@@ -0,0 +1,67 @@
+// 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.AlwaysNotNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * scalar function Ipv4StringToNumOrDefault
+ */
+public class Ipv4StringToNumOrDefault extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNotNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(StringType.INSTANCE));
+
+    public Ipv4StringToNumOrDefault(Expression arg0) {
+        super("ipv4stringtonumordefault", arg0);
+    }
+
+    @Override
+    public Ipv4StringToNumOrDefault withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1,
+                "ipv4stringtonumordefault accept 1 args, but got %s (%s)",
+                children.size(),
+                children);
+        return new Ipv4StringToNumOrDefault(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitIpv4StringToNumOrDefault(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrNull.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrNull.java
new file mode 100644
index 00000000000..618593a303a
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ipv4StringToNumOrNull.java
@@ -0,0 +1,67 @@
+// 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.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * scalar function Ipv4StringToNumOrNull
+ */
+public class Ipv4StringToNumOrNull extends ScalarFunction
+        implements BinaryExpression, ExplicitlyCastableSignature, AlwaysNullable {
+
+    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+            FunctionSignature.ret(BigIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT),
+            FunctionSignature.ret(BigIntType.INSTANCE).args(StringType.INSTANCE));
+
+    public Ipv4StringToNumOrNull(Expression arg0) {
+        super("ipv4stringtonumornull", arg0);
+    }
+
+    @Override
+    public Ipv4StringToNumOrNull withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() == 1,
+                "ipv4stringtonumornull accept 1 args, but got %s (%s)",
+                children.size(),
+                children);
+        return new Ipv4StringToNumOrNull(children.get(0));
+    }
+
+    @Override
+    public List<FunctionSignature> getSignatures() {
+        return SIGNATURES;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitIpv4StringToNumOrNull(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 e94aae003c0..20ba455a65b 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
@@ -186,6 +186,9 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNum;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrDefault;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4StringToNumOrNull;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonArray;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract;
@@ -1096,6 +1099,18 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(ipv4NumToString, context);
     }
 
+    default R visitIpv4StringToNum(Ipv4StringToNum ipv4StringToNum, C context) {
+        return visitScalarFunction(ipv4StringToNum, context);
+    }
+
+    default R visitIpv4StringToNumOrDefault(Ipv4StringToNumOrDefault ipv4StringToNumOrDefault, C context) {
+        return visitScalarFunction(ipv4StringToNumOrDefault, context);
+    }
+
+    default R visitIpv4StringToNumOrNull(Ipv4StringToNumOrNull ipv4StringToNumOrNull, C context) {
+        return visitScalarFunction(ipv4StringToNumOrNull, context);
+    }
+
     default R visitJsonArray(JsonArray jsonArray, C context) {
         return visitScalarFunction(jsonArray, context);
     }
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 77cd90ca3e2..4634604f56c 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -2005,6 +2005,12 @@ visible_functions = {
         [['ipv4numtostring','inet_ntoa'], 'VARCHAR', ['SMALLINT'], 'ALWAYS_NULLABLE'],
         [['ipv4numtostring','inet_ntoa'], 'VARCHAR', ['INT'], 'ALWAYS_NULLABLE'],
         [['ipv4numtostring','inet_ntoa'], 'VARCHAR', ['BIGINT'], 'ALWAYS_NULLABLE'],
+        [['ipv4stringtonum','inet_aton'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'],
+        [['ipv4stringtonum','inet_aton'], 'BIGINT', ['STRING'], 'ALWAYS_NOT_NULLABLE'],
+        [['ipv4stringtonumordefault'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NOT_NULLABLE'],
+        [['ipv4stringtonumordefault'], 'BIGINT', ['STRING'], 'ALWAYS_NOT_NULLABLE'],
+        [['ipv4stringtonumornull'], 'BIGINT', ['VARCHAR'], 'ALWAYS_NULLABLE'],
+        [['ipv4stringtonumornull'], 'BIGINT', ['STRING'], 'ALWAYS_NULLABLE'],        
     ],
 
     "NonNullalbe": [
diff --git a/regression-test/data/nereids_function_p0/ip_functions.out b/regression-test/data/nereids_function_p0/ip_functions.out
index fbe1c8cc6db..8a3254a993b 100644
--- a/regression-test/data/nereids_function_p0/ip_functions.out
+++ b/regression-test/data/nereids_function_p0/ip_functions.out
@@ -23,3 +23,14 @@
 -- !ip8 --
 192.168.0.1
 
+-- !ip9 --
+2130706433
+
+-- !ip10 --
+\N
+
+-- !ip11 --
+0
+
+-- !ip12 --
+3232235521
diff --git a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out
index 810ecbdd96c..5005a68cd46 100644
--- a/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out
+++ b/regression-test/data/query_p0/sql_functions/ip_functions/test_ip_functions.out
@@ -9,3 +9,15 @@
 
 -- !sql --
 192.168.0.1
+
+-- !sql --
+2130706433
+
+-- !sql --
+\N
+
+-- !sql --
+0
+
+-- !sql --
+3232235521
\ No newline at end of file
diff --git a/regression-test/suites/nereids_function_p0/ip_functions.groovy b/regression-test/suites/nereids_function_p0/ip_functions.groovy
index 8ec88569ce6..5b0588e8fdd 100644
--- a/regression-test/suites/nereids_function_p0/ip_functions.groovy
+++ b/regression-test/suites/nereids_function_p0/ip_functions.groovy
@@ -27,4 +27,8 @@ suite("ip_functions") {
     qt_ip7 "SELECT inet_ntoa(4294967298);"
     qt_ip8 "SELECT inet_ntoa(3232235521);"
 
+    qt_ip9  "SELECT ipv4stringtonum('127.0.0.1');"
+    qt_ip10 "SELECT ipv4stringtonumornull('');"
+    qt_ip11 "SELECT ipv4stringtonumordefault('');"
+    qt_ip12 "SELECT inet_aton('192.168.0.1');"
 }
\ No newline at end of file
diff --git a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy
index 2086a294af5..5d222f544a2 100644
--- a/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy
+++ b/regression-test/suites/query_p0/sql_functions/ip_functions/test_ip_functions.groovy
@@ -22,4 +22,8 @@ suite("test_ip_functions") {
     qt_sql "SELECT ipv4numtostring(4294967298);"
     qt_sql "SELECT inet_ntoa(3232235521);"
 
+    qt_sql "SELECT ipv4stringtonum('127.0.0.1');"
+    qt_sql "SELECT ipv4stringtonumornull('');"
+    qt_sql "SELECT ipv4stringtonumordefault('');"
+    qt_sql "SELECT inet_aton('192.168.0.1');"
 }


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