You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2024/01/30 12:19:01 UTC

(doris) 04/05: [fix](datatype) fix bugs for IPv4/v6 datatype and add some basic regression test cases (#30261)

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

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

commit 6b172e9003a1901e34081a059595f904e9cd400b
Author: yangshijie <sj...@zju.edu.cn>
AuthorDate: Tue Jan 30 17:29:04 2024 +0800

    [fix](datatype) fix bugs for IPv4/v6 datatype and add some basic regression test cases (#30261)
---
 be/src/exec/olap_common.h                          |   6 +
 be/src/olap/types.h                                |  14 +--
 be/src/runtime/primitive_type.h                    |   4 +-
 be/src/runtime/runtime_predicate.cpp               |   8 ++
 be/src/runtime/runtime_predicate.h                 |  10 ++
 .../aggregate_function_min_max.cpp                 |  10 ++
 be/src/vec/columns/column_vector.cpp               |   3 +-
 be/src/vec/columns/column_vector.h                 |  21 ++++
 be/src/vec/core/field.h                            |  20 +++-
 be/src/vec/core/types.h                            |  19 ++-
 be/src/vec/core/wide_integer_impl.h                |   3 +-
 be/src/vec/data_types/convert_field_to_type.cpp    |   1 +
 be/src/vec/data_types/data_type_number_base.cpp    |   6 +-
 .../data_types/serde/data_type_number_serde.cpp    |   9 +-
 .../vec/data_types/serde/data_type_number_serde.h  |   2 +
 be/src/vec/functions/functions_comparison.h        |   4 +
 be/src/vec/io/io_helper.h                          |   6 +-
 be/src/vec/olap/olap_data_convertor.cpp            |   4 +-
 .../sql-manual/sql-reference/Data-Types/IPV4.md    |   1 -
 .../sql-manual/sql-reference/Data-Types/IPV6.md    |   3 +-
 .../sql-manual/sql-reference/Data-Types/IPV4.md    |   1 -
 .../sql-manual/sql-reference/Data-Types/IPV6.md    |   3 +-
 .../data/datatype_p0/ip/test_ip_basic.out          | 131 +++++++++++++++++++++
 .../suites/datatype_p0/ip/test_ip_basic.groovy     | 106 +++++++++++++++++
 24 files changed, 363 insertions(+), 32 deletions(-)

diff --git a/be/src/exec/olap_common.h b/be/src/exec/olap_common.h
index cdca939c6ed..0ea771deb88 100644
--- a/be/src/exec/olap_common.h
+++ b/be/src/exec/olap_common.h
@@ -42,6 +42,8 @@
 #include "runtime/type_limit.h"
 #include "vec/core/types.h"
 #include "vec/io/io_helper.h"
+#include "vec/runtime/ipv4_value.h"
+#include "vec/runtime/ipv6_value.h"
 #include "vec/runtime/vdatetime_value.h"
 
 namespace doris {
@@ -68,6 +70,10 @@ std::string cast_to_string(T value, int scale) {
         std::stringstream ss;
         ss << buf;
         return ss.str();
+    } else if constexpr (primitive_type == TYPE_IPV4) {
+        return IPv4Value::to_string(value);
+    } else if constexpr (primitive_type == TYPE_IPV6) {
+        return IPv6Value::to_string(value);
     } else {
         return boost::lexical_cast<std::string>(value);
     }
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 0b2be2c38bb..74aa86fc49e 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -85,7 +85,7 @@ public:
 
     virtual void direct_copy(void* dest, const void* src) const = 0;
 
-    // Use only in zone map to cut data.StringParser::string_to_unsigned_int<uint32_t>
+    // Use only in zone map to cut data.
     virtual void direct_copy_may_cut(void* dest, const void* src) const = 0;
 
     virtual Status from_string(void* buf, const std::string& scan_key, const int precision = 0,
@@ -790,8 +790,6 @@ struct BaseFieldTypeTraits : public CppTypeTraits<field_type> {
     static inline CppType get_cpp_type_value(const void* address) {
         if constexpr (field_type == FieldType::OLAP_FIELD_TYPE_LARGEINT) {
             return get_int128_from_unalign(address);
-        } else if constexpr (field_type == FieldType::OLAP_FIELD_TYPE_IPV6) {
-            return get_int128_from_unalign(address);
         }
         return *reinterpret_cast<const CppType*>(address);
     }
@@ -1011,27 +1009,27 @@ struct FieldTypeTraits<FieldType::OLAP_FIELD_TYPE_IPV6>
         : public BaseFieldTypeTraits<FieldType::OLAP_FIELD_TYPE_IPV6> {
     static Status from_string(void* buf, const std::string& scan_key, const int precision,
                               const int scale) {
-        int128_t value;
+        uint128_t value;
         if (!IPv6Value::from_string(value, scan_key)) {
             return Status::Error<ErrorCode::INVALID_ARGUMENT>(
                     "FieldTypeTraits<OLAP_FIELD_TYPE_IPV6>::from_string meet PARSE_FAILURE");
         }
-        memcpy(buf, &value, sizeof(int128_t));
+        memcpy(buf, &value, sizeof(uint128_t));
         return Status::OK();
     }
 
     static std::string to_string(const void* src) {
-        int128_t value = *reinterpret_cast<const int128_t*>(src);
+        uint128_t value = *reinterpret_cast<const uint128_t*>(src);
         IPv6Value ipv6_value(value);
         return ipv6_value.to_string();
     }
 
     static void set_to_max(void* buf) {
-        *reinterpret_cast<int128_t*>(buf) = -1; // ::1
+        *reinterpret_cast<int128_t*>(buf) = -1; // ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
     }
 
     static void set_to_min(void* buf) {
-        *reinterpret_cast<int128_t*>(buf) = 0; // ::
+        *reinterpret_cast<uint128_t*>(buf) = 0; // ::
     }
 };
 
diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h
index 6040f73388b..d6a4354c9a6 100644
--- a/be/src/runtime/primitive_type.h
+++ b/be/src/runtime/primitive_type.h
@@ -243,13 +243,13 @@ struct PrimitiveTypeTraits<TYPE_LARGEINT> {
 template <>
 struct PrimitiveTypeTraits<TYPE_IPV4> {
     using CppType = vectorized::IPv4;
-    using StorageFieldType = uint32_t;
+    using StorageFieldType = CppType;
     using ColumnType = vectorized::ColumnIPv4;
 };
 template <>
 struct PrimitiveTypeTraits<TYPE_IPV6> {
     using CppType = vectorized::IPv6;
-    using StorageFieldType = uint64_t;
+    using StorageFieldType = CppType;
     using ColumnType = vectorized::ColumnIPv6;
 };
 template <>
diff --git a/be/src/runtime/runtime_predicate.cpp b/be/src/runtime/runtime_predicate.cpp
index 627e22364e5..5141b65978d 100644
--- a/be/src/runtime/runtime_predicate.cpp
+++ b/be/src/runtime/runtime_predicate.cpp
@@ -115,6 +115,14 @@ Status RuntimePredicate::init(const PrimitiveType type, const bool nulls_first)
         _get_value_fn = get_decimal256_value;
         break;
     }
+    case PrimitiveType::TYPE_IPV4: {
+        _get_value_fn = get_ipv4_value;
+        break;
+    }
+    case PrimitiveType::TYPE_IPV6: {
+        _get_value_fn = get_ipv6_value;
+        break;
+    }
     default:
         return Status::InvalidArgument("unsupported runtime predicate type {}", type);
     }
diff --git a/be/src/runtime/runtime_predicate.h b/be/src/runtime/runtime_predicate.h
index 80742fa66dc..297d90979ec 100644
--- a/be/src/runtime/runtime_predicate.h
+++ b/be/src/runtime/runtime_predicate.h
@@ -179,6 +179,16 @@ private:
         auto v = field.get<DecimalField<Decimal256>>();
         return cast_to_string<TYPE_DECIMAL256, ValueType>(v.get_value(), v.get_scale());
     }
+
+    static std::string get_ipv4_value(const Field& field) {
+        using ValueType = typename PrimitiveTypeTraits<TYPE_IPV4>::CppType;
+        return cast_to_string<TYPE_IPV4, ValueType>(field.get<ValueType>(), 0);
+    }
+
+    static std::string get_ipv6_value(const Field& field) {
+        using ValueType = typename PrimitiveTypeTraits<TYPE_IPV6>::CppType;
+        return cast_to_string<TYPE_IPV6, ValueType>(field.get<ValueType>(), 0);
+    }
 };
 
 } // namespace vectorized
diff --git a/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
index 27afac7789c..8aa8850a314 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
+++ b/be/src/vec/aggregate_functions/aggregate_function_min_max.cpp
@@ -68,6 +68,16 @@ AggregateFunctionPtr create_aggregate_function_single_value(const String& name,
                 AggregateFunctionsSingleValue<Data<SingleValueDataFixed<UInt64>>>>(
                 argument_types, result_is_nullable);
     }
+    if (which.idx == TypeIndex::IPv4) {
+        return creator_without_type::create<
+                AggregateFunctionsSingleValue<Data<SingleValueDataFixed<IPv4>>>>(
+                argument_types, result_is_nullable);
+    }
+    if (which.idx == TypeIndex::IPv6) {
+        return creator_without_type::create<
+                AggregateFunctionsSingleValue<Data<SingleValueDataFixed<IPv6>>>>(
+                argument_types, result_is_nullable);
+    }
     return nullptr;
 }
 
diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp
index 7329f898044..05dd3d2ddeb 100644
--- a/be/src/vec/columns/column_vector.cpp
+++ b/be/src/vec/columns/column_vector.cpp
@@ -577,7 +577,7 @@ void ColumnVector<T>::replace_column_null_data(const uint8_t* __restrict null_ma
 /// Explicit template instantiations - to avoid code bloat in headers.
 template class ColumnVector<UInt8>;
 template class ColumnVector<UInt16>;
-template class ColumnVector<UInt32>;
+template class ColumnVector<UInt32>; // IPv4
 template class ColumnVector<UInt64>;
 template class ColumnVector<UInt128>;
 template class ColumnVector<Int8>;
@@ -587,4 +587,5 @@ template class ColumnVector<Int64>;
 template class ColumnVector<Int128>;
 template class ColumnVector<Float32>;
 template class ColumnVector<Float64>;
+template class ColumnVector<IPv6>; // IPv6
 } // namespace doris::vectorized
diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h
index 384a8daa1c7..c75d7db6054 100644
--- a/be/src/vec/columns/column_vector.h
+++ b/be/src/vec/columns/column_vector.h
@@ -67,6 +67,10 @@ namespace doris::vectorized {
   * Integer values are compared as usual.
   * Floating-point numbers are compared this way that NaNs always end up at the end
   *  (if you don't do this, the sort would not work at all).
+  * Due to IPv4 being a Little-Endian storage, comparing UInt32 is equivalent to comparing IPv4.
+  * However, IPv6 is a Big-Endian storage, and comparing IPv6 is not equivalent to comparing uint128_t.
+  * So we should use std::memcmp to start comparing from low bytes to high bytes.
+  *  (e.g. :: < ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff)
   */
 template <typename T>
 struct CompareHelper {
@@ -126,6 +130,23 @@ struct CompareHelper<Float32> : public FloatCompareHelper<Float32> {};
 template <>
 struct CompareHelper<Float64> : public FloatCompareHelper<Float64> {};
 
+struct IPv6CompareHelper {
+    static bool less(IPv6 a, IPv6 b, int /*nan_direction_hint*/) {
+        return std::memcmp(&a, &b, sizeof(IPv6)) < 0;
+    }
+
+    static bool greater(IPv6 a, IPv6 b, int /*nan_direction_hint*/) {
+        return std::memcmp(&a, &b, sizeof(IPv6)) > 0;
+    }
+
+    static int compare(IPv6 a, IPv6 b, int /*nan_direction_hint*/) {
+        return std::memcmp(&a, &b, sizeof(IPv6));
+    }
+};
+
+template <>
+struct CompareHelper<IPv6> : public IPv6CompareHelper {};
+
 /** A template for columns that use a simple array to store.
  */
 template <typename T>
diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h
index 7897f990ae8..6361df6e788 100644
--- a/be/src/vec/core/field.h
+++ b/be/src/vec/core/field.h
@@ -341,6 +341,7 @@ public:
             UInt128 = 4,
             Int128 = 5,
             FixedLengthObject = 6,
+            IPv6 = 7,
 
             /// Non-POD types.
 
@@ -359,7 +360,7 @@ public:
             HyperLogLog = 28,
             QuantileState = 29,
             Int256 = 30,
-            Decimal256 = 31,
+            Decimal256 = 31
         };
 
         static const int MIN_NON_POD = 16;
@@ -408,6 +409,8 @@ public:
                 return "HyperLogLog";
             case QuantileState:
                 return "QuantileState";
+            case IPv6:
+                return "IPv6";
             default:
                 LOG(FATAL) << "type not supported, type=" << Types::to_string(which);
                 break;
@@ -580,6 +583,8 @@ public:
             return get<Int64>() <=> rhs.get<Int64>();
         case Types::Int128:
             return get<Int128>() <=> rhs.get<Int128>();
+        case Types::IPv6:
+            return get<IPv6>() <=> rhs.get<IPv6>();
         case Types::Float64:
             return get<Float64>() < rhs.get<Float64>()    ? std::strong_ordering::less
                    : get<Float64>() == rhs.get<Float64>() ? std::strong_ordering::equal
@@ -622,6 +627,9 @@ public:
         case Types::Int128:
             f(field.template get<Int128>());
             return;
+        case Types::IPv6:
+            f(field.template get<IPv6>());
+            return;
         case Types::Float64:
             f(field.template get<Float64>());
             return;
@@ -675,7 +683,7 @@ public:
 
 private:
     std::aligned_union_t<DBMS_MIN_FIELD_SIZE - sizeof(Types::Which), Null, UInt64, UInt128, Int64,
-                         Int128, Float64, String, JsonbField, Array, Tuple, Map, VariantMap,
+                         Int128, IPv6, Float64, String, JsonbField, Array, Tuple, Map, VariantMap,
                          DecimalField<Decimal32>, DecimalField<Decimal64>,
                          DecimalField<Decimal128V2>, DecimalField<Decimal128V3>,
                          DecimalField<Decimal256>, BitmapValue, HyperLogLog, QuantileState>
@@ -835,6 +843,10 @@ struct Field::TypeToEnum<Float64> {
     static constexpr Types::Which value = Types::Float64;
 };
 template <>
+struct Field::TypeToEnum<IPv6> {
+    static constexpr Types::Which value = Types::IPv6;
+};
+template <>
 struct Field::TypeToEnum<String> {
     static constexpr Types::Which value = Types::String;
 };
@@ -919,6 +931,10 @@ struct Field::EnumToType<Field::Types::Float64> {
     using Type = Float64;
 };
 template <>
+struct Field::EnumToType<Field::Types::IPv6> {
+    using Type = IPv6;
+};
+template <>
 struct Field::EnumToType<Field::Types::String> {
     using Type = String;
 };
diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h
index 82bc67a8fc5..8899b6ce017 100644
--- a/be/src/vec/core/types.h
+++ b/be/src/vec/core/types.h
@@ -296,10 +296,21 @@ struct TypeId<String> {
 /// Not a data type in database, defined just for convenience.
 using Strings = std::vector<String>;
 
-using Int128 = __int128;
+using IPv4 = uint32_t;
+using IPv6 = uint128_t;
+
+template <>
+inline constexpr bool IsNumber<IPv6> = true;
+template <>
+struct TypeName<IPv6> {
+    static const char* get() { return "IPv6"; }
+};
+template <>
+struct TypeId<IPv6> {
+    static constexpr const TypeIndex value = TypeIndex::IPv6;
+};
 
-using IPv4 = UInt32;
-using IPv6 = Int128;
+using Int128 = __int128;
 
 template <>
 inline constexpr bool IsNumber<Int128> = true;
@@ -514,6 +525,7 @@ struct Decimal {
     explicit(IsInt256) Decimal(Int32 value) noexcept : value(value) {}
     explicit(IsInt256) Decimal(Int64 value) noexcept : value(value) {}
     explicit(IsInt256) Decimal(Int128 value) noexcept : value(value) {}
+    explicit(IsInt256) Decimal(IPv6 value) noexcept : value(value) {}
     explicit(IsInt256) Decimal(wide::Int256 value) noexcept : value(value) {}
     explicit(IsInt256) Decimal(UInt64 value) noexcept : value(value) {}
     explicit(IsInt256) Decimal(UInt32 value) noexcept : value(value) {}
@@ -658,6 +670,7 @@ struct Decimal128V3 : public Decimal<Int128> {
 
     DECLARE_NUMERIC_CTOR(wide::Int256)
     DECLARE_NUMERIC_CTOR(Int128)
+    DECLARE_NUMERIC_CTOR(IPv6)
     DECLARE_NUMERIC_CTOR(Int32)
     DECLARE_NUMERIC_CTOR(Int64)
     DECLARE_NUMERIC_CTOR(UInt32)
diff --git a/be/src/vec/core/wide_integer_impl.h b/be/src/vec/core/wide_integer_impl.h
index 20f0f9d4937..2eb9381bd37 100644
--- a/be/src/vec/core/wide_integer_impl.h
+++ b/be/src/vec/core/wide_integer_impl.h
@@ -257,7 +257,8 @@ struct integer<Bits, Signed>::_impl {
     template <typename Integral>
     constexpr static void wide_integer_from_builtin(integer<Bits, Signed>& self,
                                                     Integral rhs) noexcept {
-        if constexpr (std::is_same_v<Integral, __int128>) {
+        if constexpr (std::is_same_v<Integral, __int128> ||
+                      std::is_same_v<Integral, unsigned __int128>) {
             self.items[little(0)] = rhs;
             self.items[little(1)] = rhs >> 64;
             if (rhs < 0) {
diff --git a/be/src/vec/data_types/convert_field_to_type.cpp b/be/src/vec/data_types/convert_field_to_type.cpp
index 0196d5ede76..3717abc74e1 100644
--- a/be/src/vec/data_types/convert_field_to_type.cpp
+++ b/be/src/vec/data_types/convert_field_to_type.cpp
@@ -117,6 +117,7 @@ public:
     void operator()(const Int128& x, JsonbWriter* writer) const {
         writer->writeInt128(int128_t(x));
     }
+    void operator()(const IPv6& x, JsonbWriter* writer) const { writer->writeInt128(int128_t(x)); }
     void operator()(const Int64& x, JsonbWriter* writer) const { writer->writeInt64(x); }
     void operator()(const Float64& x, JsonbWriter* writer) const { writer->writeDouble(x); }
     void operator()(const String& x, JsonbWriter* writer) const {
diff --git a/be/src/vec/data_types/data_type_number_base.cpp b/be/src/vec/data_types/data_type_number_base.cpp
index 662b8e18734..29032915999 100644
--- a/be/src/vec/data_types/data_type_number_base.cpp
+++ b/be/src/vec/data_types/data_type_number_base.cpp
@@ -147,7 +147,8 @@ std::string DataTypeNumberBase<T>::to_string(const IColumn& column, size_t row_n
     ColumnPtr ptr = result.first;
     row_num = result.second;
 
-    if constexpr (std::is_same<T, __int128_t>::value || std::is_same<T, UInt128>::value) {
+    if constexpr (std::is_same<T, int128_t>::value || std::is_same<T, uint128_t>::value ||
+                  std::is_same<T, UInt128>::value) {
         return int128_to_string(assert_cast<const ColumnVector<T>&>(*ptr).get_element(row_num));
     } else if constexpr (std::is_integral<T>::value) {
         return std::to_string(assert_cast<const ColumnVector<T>&>(*ptr).get_element(row_num));
@@ -264,7 +265,7 @@ bool DataTypeNumberBase<T>::is_value_represented_by_unsigned_integer() const {
 /// Explicit template instantiations - to avoid code bloat in headers.
 template class DataTypeNumberBase<UInt8>;
 template class DataTypeNumberBase<UInt16>;
-template class DataTypeNumberBase<UInt32>;
+template class DataTypeNumberBase<UInt32>; // IPv4
 template class DataTypeNumberBase<UInt64>;
 template class DataTypeNumberBase<UInt128>;
 template class DataTypeNumberBase<Int8>;
@@ -274,5 +275,6 @@ template class DataTypeNumberBase<Int64>;
 template class DataTypeNumberBase<Int128>;
 template class DataTypeNumberBase<Float32>;
 template class DataTypeNumberBase<Float64>;
+template class DataTypeNumberBase<IPv6>; // IPv6
 
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/serde/data_type_number_serde.cpp b/be/src/vec/data_types/serde/data_type_number_serde.cpp
index e9dd7ac094a..5d34ea92ae7 100644
--- a/be/src/vec/data_types/serde/data_type_number_serde.cpp
+++ b/be/src/vec/data_types/serde/data_type_number_serde.cpp
@@ -64,8 +64,8 @@ using DORIS_NUMERIC_ARROW_BUILDER =
                 arrow::UInt16Builder, Int16, arrow::Int16Builder, UInt32, arrow::UInt32Builder,
                 Int32, arrow::Int32Builder, UInt64, arrow::UInt64Builder, Int64,
                 arrow::Int64Builder, UInt128, arrow::FixedSizeBinaryBuilder, Int128,
-                arrow::FixedSizeBinaryBuilder, Float32, arrow::FloatBuilder, Float64,
-                arrow::DoubleBuilder, void,
+                arrow::FixedSizeBinaryBuilder, IPv6, arrow::FixedSizeBinaryBuilder, Float32,
+                arrow::FloatBuilder, Float64, arrow::DoubleBuilder, void,
                 void // Add this line to represent the end of the TypeMap
                 >;
 
@@ -84,7 +84,7 @@ void DataTypeNumberSerDe<T>::write_column_to_arrow(const IColumn& column, const
                                      end - start,
                                      reinterpret_cast<const uint8_t*>(arrow_null_map_data)),
                 column.get_name(), array_builder->type()->name());
-    } else if constexpr (std::is_same_v<T, Int128>) {
+    } else if constexpr (std::is_same_v<T, Int128> || std::is_same_v<T, IPv6>) {
         auto& string_builder = assert_cast<arrow::StringBuilder&>(*array_builder);
         for (size_t i = start; i < end; ++i) {
             auto& data_value = col_data[i];
@@ -341,7 +341,7 @@ Status DataTypeNumberSerDe<T>::write_column_to_orc(const std::string& timezone,
 /// Explicit template instantiations - to avoid code bloat in headers.
 template class DataTypeNumberSerDe<UInt8>;
 template class DataTypeNumberSerDe<UInt16>;
-template class DataTypeNumberSerDe<UInt32>;
+template class DataTypeNumberSerDe<UInt32>; // IPv4
 template class DataTypeNumberSerDe<UInt64>;
 template class DataTypeNumberSerDe<UInt128>;
 template class DataTypeNumberSerDe<Int8>;
@@ -351,5 +351,6 @@ template class DataTypeNumberSerDe<Int64>;
 template class DataTypeNumberSerDe<Int128>;
 template class DataTypeNumberSerDe<Float32>;
 template class DataTypeNumberSerDe<Float64>;
+template class DataTypeNumberSerDe<IPv6>; // IPv6
 } // namespace vectorized
 } // namespace doris
\ No newline at end of file
diff --git a/be/src/vec/data_types/serde/data_type_number_serde.h b/be/src/vec/data_types/serde/data_type_number_serde.h
index 806839c1e0f..cea3d451707 100644
--- a/be/src/vec/data_types/serde/data_type_number_serde.h
+++ b/be/src/vec/data_types/serde/data_type_number_serde.h
@@ -48,6 +48,8 @@ class Arena;
 //  DataTypeTime => T:Float64
 //  DataTypeDate => T:Int64
 //  DataTypeDateTime => T:Int64
+//  IPv4 => T:UInt32
+//  IPv6 => T:uint128_t
 template <typename T>
 class DataTypeNumberSerDe : public DataTypeSerDe {
     static_assert(IsNumber<T>);
diff --git a/be/src/vec/functions/functions_comparison.h b/be/src/vec/functions/functions_comparison.h
index 6a0f3aa634d..09597336c25 100644
--- a/be/src/vec/functions/functions_comparison.h
+++ b/be/src/vec/functions/functions_comparison.h
@@ -343,6 +343,7 @@ private:
                 execute_num_right_type<T0, Int32>(block, result, col_left, col_right_untyped) ||
                 execute_num_right_type<T0, Int64>(block, result, col_left, col_right_untyped) ||
                 execute_num_right_type<T0, Int128>(block, result, col_left, col_right_untyped) ||
+                execute_num_right_type<T0, IPv6>(block, result, col_left, col_right_untyped) ||
                 execute_num_right_type<T0, Float32>(block, result, col_left, col_right_untyped) ||
                 execute_num_right_type<T0, Float64>(block, result, col_left, col_right_untyped))
                 return true;
@@ -371,6 +372,8 @@ private:
                                                         col_right_untyped) ||
                 execute_num_const_right_type<T0, Int128>(block, result, col_left_const,
                                                          col_right_untyped) ||
+                execute_num_const_right_type<T0, IPv6>(block, result, col_left_const,
+                                                       col_right_untyped) ||
                 execute_num_const_right_type<T0, Float32>(block, result, col_left_const,
                                                           col_right_untyped) ||
                 execute_num_const_right_type<T0, Float64>(block, result, col_left_const,
@@ -587,6 +590,7 @@ public:
                                                col_right_untyped) ||
                   execute_num_left_type<Int128>(block, result, col_left_untyped,
                                                 col_right_untyped) ||
+                  execute_num_left_type<IPv6>(block, result, col_left_untyped, col_right_untyped) ||
                   execute_num_left_type<Float32>(block, result, col_left_untyped,
                                                  col_right_untyped) ||
                   execute_num_left_type<Float64>(block, result, col_left_untyped,
diff --git a/be/src/vec/io/io_helper.h b/be/src/vec/io/io_helper.h
index 81425e7bc53..1f7a8ffdf18 100644
--- a/be/src/vec/io/io_helper.h
+++ b/be/src/vec/io/io_helper.h
@@ -45,7 +45,11 @@ static constexpr size_t DEFAULT_MAX_STRING_SIZE = 1073741824; // 1GB
 static constexpr size_t DEFAULT_MAX_JSON_SIZE = 1073741824;   // 1GB
 static constexpr auto WRITE_HELPERS_MAX_INT_WIDTH = 40U;
 
-inline std::string int128_to_string(__int128_t value) {
+inline std::string int128_to_string(int128_t value) {
+    return fmt::format(FMT_COMPILE("{}"), value);
+}
+
+inline std::string int128_to_string(uint128_t value) {
     return fmt::format(FMT_COMPILE("{}"), value);
 }
 
diff --git a/be/src/vec/olap/olap_data_convertor.cpp b/be/src/vec/olap/olap_data_convertor.cpp
index d2f1d431daa..3a130a638b8 100644
--- a/be/src/vec/olap/olap_data_convertor.cpp
+++ b/be/src/vec/olap/olap_data_convertor.cpp
@@ -158,10 +158,10 @@ OlapBlockDataConvertor::create_olap_column_data_convertor(const TabletColumn& co
         return std::make_unique<OlapColumnDataConvertorSimple<vectorized::Int128>>();
     }
     case FieldType::OLAP_FIELD_TYPE_IPV4: {
-        return std::make_unique<OlapColumnDataConvertorSimple<vectorized::UInt32>>();
+        return std::make_unique<OlapColumnDataConvertorSimple<vectorized::IPv4>>();
     }
     case FieldType::OLAP_FIELD_TYPE_IPV6: {
-        return std::make_unique<OlapColumnDataConvertorSimple<vectorized::Int128>>();
+        return std::make_unique<OlapColumnDataConvertorSimple<vectorized::IPv6>>();
     }
     case FieldType::OLAP_FIELD_TYPE_FLOAT: {
         return std::make_unique<OlapColumnDataConvertorSimple<vectorized::Float32>>();
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md
index 07ab836e4eb..477371ecc2a 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV4.md
@@ -34,7 +34,6 @@ IPV4
 
 ### description
 
-IPV4
 IPv4 type, stored in the form of UInt32 in 4 bytes, used to represent IPv4 addresses.
 The range of values is ['0.0.0.0', '255.255.255.255'].
 
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md
index dcf46813240..f877bbc2d40 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Types/IPV6.md
@@ -34,8 +34,7 @@ IPV6
 
 ### description
 
-IPV6
-IPv6 type, stored in Int128 format in 16 bytes, used to represent IPv6 addresses.
+IPv6 type, stored in UInt128 format in 16 bytes, used to represent IPv6 addresses.
 The range of values is ['::', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'].
 
 `Inputs that exceed the value range or have invalid format will return NULL`
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md
index 2766a20580e..f44eb195e0d 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV4.md
@@ -34,7 +34,6 @@ IPV4
 
 ### description
 
-IPV4
 IPv4类型,以UInt32的形式存储在4个字节中,用于表示IPv4地址。
 取值范围是 ['0.0.0.0', '255.255.255.255']。
 
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md
index 678b404a672..591ad1eb125 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Types/IPV6.md
@@ -34,8 +34,7 @@ IPV6
 
 ### description
 
-IPV6
-IPv6类型,以Int128的形式存储在16个字节中,用于表示IPv6地址。
+IPv6类型,以UInt128的形式存储在16个字节中,用于表示IPv6地址。
 取值范围是 ['::', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff']。
 
 `超出取值范围或者格式非法的输入将返回NULL`
diff --git a/regression-test/data/datatype_p0/ip/test_ip_basic.out b/regression-test/data/datatype_p0/ip/test_ip_basic.out
new file mode 100644
index 00000000000..0262959d83e
--- /dev/null
+++ b/regression-test/data/datatype_p0/ip/test_ip_basic.out
@@ -0,0 +1,131 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !sql1 --
+\N
+0.0.0.0
+127.0.0.1
+127.0.0.1
+127.0.0.1
+192.168.0.1
+192.168.0.1
+255.255.255.255
+255.255.255.255
+255.255.255.255
+255.255.255.255
+
+-- !sql2 --
+\N
+::
+::1
+::1
+2001:1b70:a1:610::b102:2
+2001:1b70:a1:610::b102:2
+2001:1b70:a1:610::b102:2
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+
+-- !sql3 --
+255.255.255.255
+255.255.255.255
+255.255.255.255
+255.255.255.255
+192.168.0.1
+192.168.0.1
+127.0.0.1
+127.0.0.1
+127.0.0.1
+0.0.0.0
+\N
+
+-- !sql4 --
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+2001:1b70:a1:610::b102:2
+2001:1b70:a1:610::b102:2
+2001:1b70:a1:610::b102:2
+::1
+::1
+::
+\N
+
+-- !sql5 --
+0.0.0.0
+
+-- !sql6 --
+::
+
+-- !sql7 --
+255.255.255.255
+
+-- !sql8 --
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+
+-- !sql9 --
+\N	0
+0.0.0.0	1
+127.0.0.1	3
+192.168.0.1	2
+255.255.255.255	4
+
+-- !sql10 --
+\N	0
+::	1
+::1	2
+2001:1b70:a1:610::b102:2	3
+ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4
+
+-- !sql11 --
+0	\N	\N
+1	0.0.0.0	0.0.0.0
+3	127.0.0.1	127.0.0.1
+2	192.168.0.1	192.168.0.1
+4	255.255.255.255	255.255.255.255
+
+-- !sql12 --
+0	\N	\N
+1	::	::
+2	::1	::1
+3	2001:1b70:a1:610::b102:2	2001:1b70:a1:610::b102:2
+4	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+
+-- !sql13 --
+1	0.0.0.0	::	1	0.0.0.0	::
+2	192.168.0.1	::1	2	192.168.0.1	::1
+2	192.168.0.1	::1	2	192.168.0.1	::1
+2	192.168.0.1	::1	2	192.168.0.1	::1
+2	192.168.0.1	::1	2	192.168.0.1	::1
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+3	127.0.0.1	2001:1b70:a1:610::b102:2	3	127.0.0.1	2001:1b70:a1:610::b102:2
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff	4	255.255.255.255	ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff
+
+-- !sql14 --
+0.0.0.0	1
+192.168.0.1	4
+127.0.0.1	9
+255.255.255.255	16
+
diff --git a/regression-test/suites/datatype_p0/ip/test_ip_basic.groovy b/regression-test/suites/datatype_p0/ip/test_ip_basic.groovy
new file mode 100644
index 00000000000..586f3d15774
--- /dev/null
+++ b/regression-test/suites/datatype_p0/ip/test_ip_basic.groovy
@@ -0,0 +1,106 @@
+
+// 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_ip_basic") {
+    sql """ DROP TABLE IF EXISTS t0 """
+    sql """ DROP TABLE IF EXISTS t1 """
+
+    sql """ SET enable_nereids_planner=true """
+    sql """ SET enable_fallback_to_original_planner=false """
+
+    sql """
+        CREATE TABLE `t0` (
+          `id` int,
+          `ip_v4` ipv4,
+          `ip_v6` ipv6
+        ) ENGINE=OLAP
+        DISTRIBUTED BY HASH(`id`) BUCKETS 4
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "enable_duplicate_without_keys_by_default" = "true"
+        );
+        """
+
+    sql """
+        CREATE TABLE `t1` (
+          `id` int,
+          `ip_v4` ipv4,
+          `ip_v6` ipv6
+        ) ENGINE=OLAP
+        DISTRIBUTED BY HASH(`id`) BUCKETS 4
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "enable_duplicate_without_keys_by_default" = "true"
+        );
+        """
+
+    sql """
+        insert into t0 values
+        (0, NULL, NULL),
+        (1, '0.0.0.0', '::'),
+        (2, '192.168.0.1', '::1'),
+        (2, '192.168.0.1', '::1'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff');
+        """
+
+    sql """
+        insert into t1 values
+        (0, NULL, NULL),
+        (1, '0.0.0.0', '::'),
+        (2, '192.168.0.1', '::1'),
+        (2, '192.168.0.1', '::1'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (3, '127.0.0.1', '2001:1b70:a1:610::b102:2'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'),
+        (4, '255.255.255.255', 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff');
+        """
+
+    // order by
+    qt_sql1 "select ip_v4 from t0 order by ip_v4"
+    qt_sql2 "select ip_v6 from t0 order by ip_v6"
+    qt_sql3 "select ip_v4 from t0 order by ip_v4 desc"
+    qt_sql4 "select ip_v6 from t0 order by ip_v6 desc"
+    qt_sql5 "select ip_v4 from t0 order by ip_v4 limit 1 offset 1"
+    qt_sql6 "select ip_v6 from t0 order by ip_v6 limit 1 offset 1"
+    qt_sql7 "select ip_v4 from t0 order by ip_v4 desc limit 1 offset 1"
+    qt_sql8 "select ip_v6 from t0 order by ip_v6 desc limit 1 offset 1"
+
+    // group by and agg
+    qt_sql9 "select ip_v4, count(ip_v4) as cnt from t0 group by ip_v4 order by ip_v4"
+    qt_sql10 "select ip_v6, count(ip_v6) as cnt from t0 group by ip_v6 order by ip_v6"
+    qt_sql11 "select count(ip_v4) as cnt, min(ip_v4), max(ip_v4) from t0 group by ip_v4 order by min(ip_v4)"
+    qt_sql12 "select count(ip_v6) as cnt, min(ip_v6), max(ip_v6) from t0 group by ip_v6 order by min(ip_v6)"
+
+    // join
+    qt_sql13 "select t0.id, t0.ip_v4, t0.ip_v6, t1.id, t1.ip_v4, t1.ip_v6 from t0 join t1 on t0.ip_v4=t1.ip_v4 and t0.ip_v6=t1.ip_v6 order by t0.id, t1.id"
+
+    // join and group by
+    qt_sql14 "select t0.ip_v4, count(*) as cnt from t0 join t1 on t0.ip_v4=t1.ip_v4 and t0.ip_v6=t1.ip_v6 group by t0.ip_v4 order by cnt"
+
+    sql "DROP TABLE t0"
+    sql "DROP TABLE t1"
+}


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