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 2023/04/09 10:08:40 UTC

[doris] branch master updated: [refactor](typesystem) using typeindex to create column instead of type name because type name is not stable (#18328)

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

yiguolei 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 f38e00b4c0 [refactor](typesystem) using typeindex to create column instead of type name because type name is not stable (#18328)
f38e00b4c0 is described below

commit f38e00b4c0cace729ad7ad1c8a079e76897e4bdb
Author: yiguolei <67...@qq.com>
AuthorDate: Sun Apr 9 18:08:31 2023 +0800

    [refactor](typesystem) using typeindex to create column instead of type name because type name is not stable (#18328)
    
    
    ---------
    
    Co-authored-by: yiguolei <yi...@gmail.com>
---
 be/src/vec/columns/column_object.cpp         |   2 +-
 be/src/vec/columns/column_struct.cpp         |   3 +-
 be/src/vec/columns/column_vector.h           |   6 +
 be/src/vec/core/types.h                      |  86 +++----
 be/src/vec/data_types/data_type_factory.cpp  |  91 ++++++++
 be/src/vec/data_types/data_type_factory.hpp  | 121 +---------
 be/src/vec/data_types/data_type_struct.cpp   |   2 +-
 be/src/vec/exec/vjdbc_connector.cpp          |  11 +-
 be/src/vec/exprs/vcast_expr.cpp              |  10 +-
 be/src/vec/exprs/vschema_change_expr.cpp     |  27 ---
 be/src/vec/functions/function_cast.h         |  15 +-
 be/test/vec/function/function_jsonb_test.cpp | 333 ++++++++++++++-------------
 12 files changed, 340 insertions(+), 367 deletions(-)

diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp
index 3b8d3a2055..1af9f3f630 100644
--- a/be/src/vec/columns/column_object.cpp
+++ b/be/src/vec/columns/column_object.cpp
@@ -66,7 +66,7 @@ size_t getNumberOfDimensions(const IDataType& type) {
 DataTypePtr get_data_type_by_column(const IColumn& column) {
     auto idx = column.get_data_type();
     if (WhichDataType(idx).is_simple()) {
-        return DataTypeFactory::instance().get(String(getTypeName(idx)));
+        return DataTypeFactory::instance().create_data_type(idx);
     }
     if (WhichDataType(idx).is_nothing()) {
         return std::make_shared<DataTypeNothing>();
diff --git a/be/src/vec/columns/column_struct.cpp b/be/src/vec/columns/column_struct.cpp
index e0663b6386..f9f25db351 100644
--- a/be/src/vec/columns/column_struct.cpp
+++ b/be/src/vec/columns/column_struct.cpp
@@ -119,7 +119,8 @@ void ColumnStruct::insert(const Field& x) {
     const auto& tuple = x.get<const Tuple&>();
     const size_t tuple_size = columns.size();
     if (tuple.size() != tuple_size) {
-        LOG(FATAL) << "Cannot insert value of different size into tuple.";
+        LOG(FATAL) << "Cannot insert value of different size into tuple. field tuple size"
+                   << tuple.size() << ", columns size " << tuple_size;
     }
 
     for (size_t i = 0; i < tuple_size; ++i) {
diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h
index f1678715c7..791893f1c4 100644
--- a/be/src/vec/columns/column_vector.h
+++ b/be/src/vec/columns/column_vector.h
@@ -295,6 +295,12 @@ public:
 
     Int64 get_int(size_t n) const override { return Int64(data[n]); }
 
+    // For example, during create column_const(1, uint8), will use NearestFieldType
+    // to cast a uint8 to int64, so that the Field is int64, but the column is created
+    // using data_type, so that T == uint8. After the field is created, it will be inserted
+    // into the column, but its type is different from column's data type, so that during column
+    // insert method, should use NearestFieldType<T> to get the Field and get it actual
+    // uint8 value and then insert into column.
     void insert(const Field& x) override {
         data.push_back(doris::vectorized::get<NearestFieldType<T>>(x));
     }
diff --git a/be/src/vec/core/types.h b/be/src/vec/core/types.h
index e2d92702b7..c88b3f3948 100644
--- a/be/src/vec/core/types.h
+++ b/be/src/vec/core/types.h
@@ -45,50 +45,52 @@ namespace vectorized {
 
 struct Null {};
 
+// The identifier should be less than int16, because castexpr using the identifier
+// instead of type name as type parameter. It will using int16 as column type.
 enum class TypeIndex {
     Nothing = 0,
-    UInt8,
-    UInt16,
-    UInt32,
-    UInt64,
-    UInt128,
-    Int8,
-    Int16,
-    Int32,
-    Int64,
-    Int128,
-    Float32,
-    Float64,
-    Date,
-    DateTime,
-    String,
-    FixedString,
-    Enum8,
-    Enum16,
-    Decimal32,
-    Decimal64,
-    Decimal128,
-    UUID,
-    Array,
-    Tuple,
-    Set,
-    Interval,
-    Nullable,
-    Function,
-    AggregateFunction,
-    LowCardinality,
-    BitMap,
-    HLL,
-    DateV2,
-    DateTimeV2,
-    TimeV2,
-    FixedLengthObject,
-    JSONB,
-    Decimal128I,
-    Map,
-    Struct,
-    VARIANT,
-    QuantileState,
+    UInt8 = 1,
+    UInt16 = 2,
+    UInt32 = 3,
+    UInt64 = 4,
+    UInt128 = 5,
+    Int8 = 6,
+    Int16 = 7,
+    Int32 = 8,
+    Int64 = 9,
+    Int128 = 10,
+    Float32 = 11,
+    Float64 = 12,
+    Date = 13,
+    DateTime = 14,
+    String = 15,
+    FixedString = 16,
+    Enum8 = 17,
+    Enum16 = 18,
+    Decimal32 = 19,
+    Decimal64 = 20,
+    Decimal128 = 21,
+    UUID = 22,
+    Array = 23,
+    Tuple = 24,
+    Set = 25,
+    Interval = 26,
+    Nullable = 27,
+    Function = 28,
+    AggregateFunction = 29,
+    LowCardinality = 30,
+    BitMap = 31,
+    HLL = 32,
+    DateV2 = 33,
+    DateTimeV2 = 34,
+    TimeV2 = 35,
+    FixedLengthObject = 36,
+    JSONB = 37,
+    Decimal128I = 38,
+    Map = 39,
+    Struct = 40,
+    VARIANT = 41,
+    QuantileState = 42,
 };
 
 struct Consted {
diff --git a/be/src/vec/data_types/data_type_factory.cpp b/be/src/vec/data_types/data_type_factory.cpp
index de893ab92b..2cb1f73aae 100644
--- a/be/src/vec/data_types/data_type_factory.cpp
+++ b/be/src/vec/data_types/data_type_factory.cpp
@@ -214,6 +214,97 @@ DataTypePtr DataTypeFactory::create_data_type(const TypeDescriptor& col_desc, bo
     return nested;
 }
 
+DataTypePtr DataTypeFactory::create_data_type(const TypeIndex& type_index, bool is_nullable) {
+    DataTypePtr nested = nullptr;
+    switch (type_index) {
+    case TypeIndex::UInt8:
+        nested = std::make_shared<vectorized::DataTypeUInt8>();
+        break;
+    case TypeIndex::Int8:
+        nested = std::make_shared<vectorized::DataTypeInt8>();
+        break;
+    case TypeIndex::UInt16:
+        nested = std::make_shared<vectorized::DataTypeUInt16>();
+        break;
+    case TypeIndex::Int16:
+        nested = std::make_shared<vectorized::DataTypeInt16>();
+        break;
+    case TypeIndex::UInt32:
+        nested = std::make_shared<vectorized::DataTypeUInt32>();
+        break;
+    case TypeIndex::Int32:
+        nested = std::make_shared<vectorized::DataTypeInt32>();
+        break;
+    case TypeIndex::UInt64:
+        nested = std::make_shared<vectorized::DataTypeUInt64>();
+        break;
+    case TypeIndex::Int64:
+        nested = std::make_shared<vectorized::DataTypeInt64>();
+        break;
+    case TypeIndex::Int128:
+        nested = std::make_shared<vectorized::DataTypeInt128>();
+        break;
+    case TypeIndex::Float32:
+        nested = std::make_shared<vectorized::DataTypeFloat32>();
+        break;
+    case TypeIndex::Float64:
+        nested = std::make_shared<vectorized::DataTypeFloat64>();
+        break;
+    case TypeIndex::Date:
+        nested = std::make_shared<vectorized::DataTypeDate>();
+        break;
+    case TypeIndex::DateV2:
+        nested = std::make_shared<vectorized::DataTypeDateV2>();
+        break;
+    case TypeIndex::DateTimeV2:
+        nested = std::make_shared<DataTypeDateTimeV2>();
+        break;
+    case TypeIndex::DateTime:
+        nested = std::make_shared<vectorized::DataTypeDateTime>();
+        break;
+    case TypeIndex::String:
+        nested = std::make_shared<vectorized::DataTypeString>();
+        break;
+    case TypeIndex::Decimal32:
+        nested = std::make_shared<DataTypeDecimal<Decimal32>>(BeConsts::MAX_DECIMAL32_PRECISION, 0);
+        break;
+    case TypeIndex::Decimal64:
+        nested = std::make_shared<DataTypeDecimal<Decimal64>>(BeConsts::MAX_DECIMAL64_PRECISION, 0);
+        break;
+    case TypeIndex::Decimal128:
+        nested = std::make_shared<DataTypeDecimal<Decimal128>>(BeConsts::MAX_DECIMAL128_PRECISION,
+                                                               0);
+        break;
+    case TypeIndex::Decimal128I:
+        nested = std::make_shared<DataTypeDecimal<Decimal128I>>(BeConsts::MAX_DECIMAL128_PRECISION,
+                                                                0);
+        break;
+    case TypeIndex::JSONB:
+        nested = std::make_shared<vectorized::DataTypeJsonb>();
+        break;
+    case TypeIndex::BitMap:
+        nested = std::make_shared<vectorized::DataTypeBitMap>();
+        break;
+    case TypeIndex::HLL:
+        nested = std::make_shared<vectorized::DataTypeHLL>();
+        break;
+    case TypeIndex::QuantileState:
+        nested = std::make_shared<vectorized::DataTypeQuantileStateDouble>();
+        break;
+    case TypeIndex::TimeV2:
+        nested = std::make_shared<vectorized::DataTypeTime>();
+        break;
+    default:
+        DCHECK(false) << "invalid typeindex:" << static_cast<int16_t>(type_index);
+        break;
+    }
+
+    if (nested && is_nullable) {
+        return std::make_shared<vectorized::DataTypeNullable>(nested);
+    }
+    return nested;
+}
+
 DataTypePtr DataTypeFactory::_create_primitive_data_type(const FieldType& type, int precision,
                                                          int scale) const {
     DataTypePtr result = nullptr;
diff --git a/be/src/vec/data_types/data_type_factory.hpp b/be/src/vec/data_types/data_type_factory.hpp
index 3118b7b64e..155beac08c 100644
--- a/be/src/vec/data_types/data_type_factory.hpp
+++ b/be/src/vec/data_types/data_type_factory.hpp
@@ -54,124 +54,12 @@ class DataTypeFactory {
 
 public:
     static DataTypeFactory& instance() {
-        static std::once_flag oc;
         static DataTypeFactory instance;
-        std::call_once(oc, []() {
-            std::unordered_map<std::string, DataTypePtr> base_type_map {
-                    {"UInt8", std::make_shared<DataTypeUInt8>()},
-                    {"UInt16", std::make_shared<DataTypeUInt16>()},
-                    {"UInt32", std::make_shared<DataTypeUInt32>()},
-                    {"UInt64", std::make_shared<DataTypeUInt64>()},
-                    {"Int8", std::make_shared<DataTypeInt8>()},
-                    {"Int16", std::make_shared<DataTypeInt16>()},
-                    {"Int32", std::make_shared<DataTypeInt32>()},
-                    {"Int64", std::make_shared<DataTypeInt64>()},
-                    {"Int128", std::make_shared<DataTypeInt128>()},
-                    {"Float32", std::make_shared<DataTypeFloat32>()},
-                    {"Float64", std::make_shared<DataTypeFloat64>()},
-                    {"Date", std::make_shared<DataTypeDate>()},
-                    {"DateV2", std::make_shared<DataTypeDateV2>()},
-                    {"DateTime", std::make_shared<DataTypeDateTime>()},
-                    {"DateTimeV2", std::make_shared<DataTypeDateTimeV2>()},
-                    {"String", std::make_shared<DataTypeString>()},
-                    {"Decimal", std::make_shared<DataTypeDecimal<Decimal128>>(27, 9)},
-                    {"Decimal32", std::make_shared<DataTypeDecimal<Decimal32>>(
-                                          BeConsts::MAX_DECIMAL32_PRECISION, 0)},
-                    {"Decimal64", std::make_shared<DataTypeDecimal<Decimal64>>(
-                                          BeConsts::MAX_DECIMAL64_PRECISION, 0)},
-                    {"Decimal128", std::make_shared<DataTypeDecimal<Decimal128>>(
-                                           BeConsts::MAX_DECIMAL128_PRECISION, 0)},
-                    {"Decimal128I", std::make_shared<DataTypeDecimal<Decimal128I>>(
-                                            BeConsts::MAX_DECIMAL128_PRECISION, 0)},
-                    {"Jsonb", std::make_shared<DataTypeJsonb>()},
-                    {"BitMap", std::make_shared<DataTypeBitMap>()},
-                    {"Hll", std::make_shared<DataTypeHLL>()},
-                    {"QuantileState", std::make_shared<DataTypeQuantileStateDouble>()},
-            };
-            for (auto const& [key, val] : base_type_map) {
-                instance.register_data_type(key, val);
-                instance.register_data_type("Array(" + key + ")",
-                                            std::make_shared<vectorized::DataTypeArray>(val));
-                instance.register_data_type(
-                        "Array(Nullable(" + key + "))",
-                        std::make_shared<vectorized::DataTypeArray>(
-                                std::make_shared<vectorized::DataTypeNullable>(val)));
-            }
-        });
         return instance;
     }
 
-    // TODO(xy): support creator to create dynamic struct type
-    DataTypePtr get(const std::string& name) { return _data_type_map[name]; }
-    // TODO(xy): support creator to create dynamic struct type
-    const std::string& get(const DataTypePtr& data_type) const {
-        auto type_ptr = data_type->is_nullable()
-                                ? ((DataTypeNullable*)(data_type.get()))->get_nested_type()
-                                : data_type;
-        for (const auto& entity : _invert_data_type_map) {
-            if (entity.first->equals(*type_ptr)) {
-                return entity.second;
-            }
-            if (is_decimal(type_ptr) && type_ptr->get_type_id() == entity.first->get_type_id()) {
-                return entity.second;
-            }
-            if (is_array(type_ptr) && is_array(entity.first)) {
-                auto nested_nullable_type_ptr =
-                        (assert_cast<const DataTypeArray*>(type_ptr.get()))->get_nested_type();
-                auto nested_nullable_entity_ptr =
-                        (assert_cast<const DataTypeArray*>(entity.first.get()))->get_nested_type();
-                // There must be nullable inside array type.
-                if (nested_nullable_type_ptr->is_nullable() &&
-                    nested_nullable_entity_ptr->is_nullable()) {
-                    auto nested_type_ptr = ((DataTypeNullable*)(nested_nullable_type_ptr.get()))
-                                                   ->get_nested_type();
-                    auto nested_entity_ptr = ((DataTypeNullable*)(nested_nullable_entity_ptr.get()))
-                                                     ->get_nested_type();
-                    if (is_decimal(nested_type_ptr) &&
-                        nested_type_ptr->get_type_id() == nested_entity_ptr->get_type_id()) {
-                        return entity.second;
-                    }
-                }
-            }
-        }
-        if (type_ptr->get_type_id() == TypeIndex::Struct ||
-            type_ptr->get_type_id() == TypeIndex::Map) {
-            DataTypeFactory::instance().register_data_type(type_ptr->get_name(), type_ptr);
-            for (const auto& entity : _invert_data_type_map) {
-                if (entity.first->equals(*type_ptr)) {
-                    return entity.second;
-                }
-            }
-        } else if (type_ptr->get_type_id() == TypeIndex::Array) {
-            // register the Array<Struct<>>/Array<Map<>>
-            auto nested_type = ((DataTypeArray*)type_ptr.get())->get_nested_type();
-            nested_type = nested_type->is_nullable()
-                                  ? ((DataTypeNullable*)(nested_type.get()))->get_nested_type()
-                                  : nested_type;
-
-            if (nested_type->get_type_id() == TypeIndex::Struct ||
-                nested_type->get_type_id() == TypeIndex::Map) {
-                auto key = nested_type->get_name();
-                auto val = nested_type;
-                DataTypeFactory::instance().register_data_type(key, val);
-                DataTypeFactory::instance().register_data_type(
-                        "Array(" + key + ")", std::make_shared<vectorized::DataTypeArray>(val));
-                DataTypeFactory::instance().register_data_type(
-                        "Array(Nullable(" + key + "))",
-                        std::make_shared<vectorized::DataTypeArray>(
-                                std::make_shared<vectorized::DataTypeNullable>(val)));
-            }
-
-            for (const auto& entity : _invert_data_type_map) {
-                if (entity.first->equals(*type_ptr)) {
-                    return entity.second;
-                }
-            }
-        }
-        return _empty_string;
-    }
-
     DataTypePtr create_data_type(const doris::Field& col_desc);
+    DataTypePtr create_data_type(const TypeIndex& type_index, bool is_nullable = false);
     DataTypePtr create_data_type(const TabletColumn& col_desc, bool is_nullable = false);
 
     DataTypePtr create_data_type(const TypeDescriptor& col_desc, bool is_nullable = true);
@@ -191,13 +79,6 @@ public:
 private:
     DataTypePtr _create_primitive_data_type(const FieldType& type, int precision, int scale) const;
 
-    void register_data_type(const std::string& name, const DataTypePtr& data_type) {
-        _data_type_map.emplace(name, data_type);
-        _invert_data_type_map.emplace_back(data_type, name);
-    }
-    // TODO: Here is a little trick here, use bimap to replace map and vector
-    DataTypeMap _data_type_map;
-    InvertedDataTypeMap _invert_data_type_map;
     std::string _empty_string;
 };
 } // namespace doris::vectorized
diff --git a/be/src/vec/data_types/data_type_struct.cpp b/be/src/vec/data_types/data_type_struct.cpp
index 9f4331f378..5e1ec60ae2 100644
--- a/be/src/vec/data_types/data_type_struct.cpp
+++ b/be/src/vec/data_types/data_type_struct.cpp
@@ -345,7 +345,7 @@ MutableColumnPtr DataTypeStruct::create_column() const {
 
 Field DataTypeStruct::get_default() const {
     size_t size = elems.size();
-    Tuple t(size);
+    Tuple t;
     for (size_t i = 0; i < size; ++i) {
         t.push_back(elems[i]->get_default());
     }
diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp
index c1aceb6a91..1beebd2620 100644
--- a/be/src/vec/exec/vjdbc_connector.cpp
+++ b/be/src/vec/exec/vjdbc_connector.cpp
@@ -643,9 +643,14 @@ Status JdbcConnector::_register_func_id(JNIEnv* env) {
 Status JdbcConnector::_cast_string_to_array(const SlotDescriptor* slot_desc, Block* block,
                                             int column_index, int rows) {
     DataTypePtr _target_data_type = slot_desc->get_data_type_ptr();
-    std::string _target_data_type_name = DataTypeFactory::instance().get(_target_data_type);
-    DataTypePtr _cast_param_data_type = std::make_shared<DataTypeString>();
-    ColumnPtr _cast_param = _cast_param_data_type->create_column_const(1, _target_data_type_name);
+    std::string _target_data_type_name = _target_data_type->get_name();
+    DataTypePtr _cast_param_data_type = std::make_shared<DataTypeInt16>();
+    ColumnPtr _cast_param = _cast_param_data_type->create_column_const(
+            1, static_cast<int16_t>(_target_data_type->is_nullable()
+                                            ? ((DataTypeNullable*)(_target_data_type.get()))
+                                                      ->get_nested_type()
+                                                      ->get_type_id()
+                                            : _target_data_type->get_type_id()));
 
     ColumnsWithTypeAndName argument_template;
     argument_template.reserve(2);
diff --git a/be/src/vec/exprs/vcast_expr.cpp b/be/src/vec/exprs/vcast_expr.cpp
index 86e942b24b..8253216de4 100644
--- a/be/src/vec/exprs/vcast_expr.cpp
+++ b/be/src/vec/exprs/vcast_expr.cpp
@@ -38,9 +38,13 @@ doris::Status VCastExpr::prepare(doris::RuntimeState* state, const doris::RowDes
     // create a const string column
     _target_data_type = _data_type;
     // TODO(xy): support return struct type name
-    _target_data_type_name = DataTypeFactory::instance().get(_target_data_type);
-    _cast_param_data_type = std::make_shared<DataTypeString>();
-    _cast_param = _cast_param_data_type->create_column_const(1, _target_data_type_name);
+    _target_data_type_name = _target_data_type->get_name();
+    // Using typeindex to indicate the datatype, not using type name because
+    // type name is not stable, but type index is stable and immutable
+    _cast_param_data_type = _target_data_type;
+    // Has to cast to int16_t or there will be compile error because there is no
+    // TypeIndexField
+    _cast_param = _cast_param_data_type->create_column_const_with_default_value(1);
 
     ColumnsWithTypeAndName argument_template;
     argument_template.reserve(2);
diff --git a/be/src/vec/exprs/vschema_change_expr.cpp b/be/src/vec/exprs/vschema_change_expr.cpp
index 75778b095b..cfe5e1c654 100644
--- a/be/src/vec/exprs/vschema_change_expr.cpp
+++ b/be/src/vec/exprs/vschema_change_expr.cpp
@@ -81,33 +81,6 @@ Status VSchemaChangeExpr::execute(VExprContext* context, doris::vectorized::Bloc
                                                                       full_base_schema_view.get()));
     }
 
-    // TODO: make sure the dynamic generated columns's types matched with schema in full_base_schema_view
-    // handle dynamic generated columns
-    // if (_full_base_schema_view && !_full_base_schema_view->empty()) {
-    //     CHECK(_is_dynamic_schema);
-    //     for (size_t i = block->columns(); i < _src_block.columns(); ++i) {
-    //         auto& column_type_name = _src_block.get_by_position(i);
-    //         // Column from schema change response
-    //         const TColumn& tcolumn =
-    //                 _full_base_schema_view->column_name_to_column[column_type_name.name];
-    //         auto original_type = vectorized::DataTypeFactory::instance().create_data_type(tcolumn);
-    //         // Detect type conflict, there may exist another load procedure, whitch has already added some columns
-    //         // but, this load detects different type, we go type conflict free path, always cast to original type
-    //         // TODO need to add type conflict abort feature
-    //         if (!column_type_name.type->equals(*original_type)) {
-    //             vectorized::ColumnPtr column_ptr;
-    //             RETURN_IF_ERROR(vectorized::schema_util::cast_column(column_type_name,
-    //                                                                  original_type, &column_ptr));
-    //             column_type_name.column = column_ptr;
-    //             column_type_name.type = original_type;
-    //         }
-    //         DCHECK(column_type_name.column != nullptr);
-    //         block->insert(vectorized::ColumnWithTypeAndName(std::move(column_type_name.column),
-    //                                                         std::move(column_type_name.type),
-    //                                                         column_type_name.name));
-    //     }
-    // }
-
     *result_column_id = _column_id;
     return Status::OK();
 }
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index f0d620f2df..4aeb3b373f 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -1856,18 +1856,7 @@ protected:
     }
 
     DataTypePtr get_return_type_impl(const ColumnsWithTypeAndName& arguments) const override {
-        const auto type_col =
-                check_and_get_column_const<ColumnString>(arguments.back().column.get());
-        DataTypePtr type;
-        if (!type_col) {
-            // only used in schema_util::cast_column
-            // use second arg as type arg
-            // since not all types are in the DatatypeFactory
-            type = arguments[1].type;
-        } else {
-            // TODO(xy): support return struct type for factory
-            type = DataTypeFactory::instance().get(type_col->get_value<String>());
-        }
+        DataTypePtr type = arguments[1].type;
         DCHECK(type != nullptr);
         bool need_to_be_nullable = false;
         // 1. from_type is nullable
@@ -1885,7 +1874,7 @@ protected:
                                 arguments[0].type->get_type_id() != TypeIndex::DateTimeV2) &&
                                (type->get_type_id() == TypeIndex::DateV2 ||
                                 type->get_type_id() == TypeIndex::DateTimeV2);
-        if (need_to_be_nullable) {
+        if (need_to_be_nullable && !type->is_nullable()) {
             return make_nullable(type);
         }
 
diff --git a/be/test/vec/function/function_jsonb_test.cpp b/be/test/vec/function/function_jsonb_test.cpp
index 7f1fd480e5..54e9117b48 100644
--- a/be/test/vec/function/function_jsonb_test.cpp
+++ b/be/test/vec/function/function_jsonb_test.cpp
@@ -1202,54 +1202,63 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
 
 TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
     std::string func_name = "CAST";
-    InputTypeSet input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::String}};
+    InputTypeSet input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::UInt8}};
 
     // cast to boolean
     DataSet data_set = {
-            {{STRING("null"), STRING("UInt8")}, Null()},
-            {{STRING("true"), STRING("UInt8")}, BOOLEAN(1)},
-            {{STRING("false"), STRING("UInt8")}, BOOLEAN(0)},
-            {{STRING("100"), STRING("UInt8")}, Null()},                        //int8
-            {{STRING("10000"), STRING("UInt8")}, Null()},                      // int16
-            {{STRING("1000000000"), STRING("UInt8")}, Null()},                 // int32
-            {{STRING("1152921504606846976"), STRING("UInt8")}, Null()},        // int64
-            {{STRING("6.18"), STRING("UInt8")}, Null()},                       // double
-            {{STRING(R"("abcd")"), STRING("UInt8")}, Null()},                  // string
-            {{STRING("{}"), STRING("UInt8")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("UInt8")}, Null()}, // object
-            {{STRING("[]"), STRING("UInt8")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("UInt8")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("UInt8")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("UInt8")},
+            {{STRING("null"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},
+            {{STRING("true"), static_cast<uint8_t>(TypeIndex::UInt8)}, BOOLEAN(1)},
+            {{STRING("false"), static_cast<uint8_t>(TypeIndex::UInt8)}, BOOLEAN(0)},
+            {{STRING("100"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},        //int8
+            {{STRING("10000"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},      // int16
+            {{STRING("1000000000"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()}, // int32
+            {{STRING("1152921504606846976"), static_cast<uint8_t>(TypeIndex::UInt8)},
+             Null()},                                                                // int64
+            {{STRING("6.18"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},      // double
+            {{STRING(R"("abcd")"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()}, // string
+            {{STRING("{}"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},        // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), static_cast<uint8_t>(TypeIndex::UInt8)},
+             Null()},                                                                 // object
+            {{STRING("[]"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()},         // empty array
+            {{STRING("[123, 456]"), static_cast<uint8_t>(TypeIndex::UInt8)}, Null()}, // int array
+            {{STRING(R"(["abc", "def"])"), static_cast<uint8_t>(TypeIndex::UInt8)},
+             Null()}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"),
+              static_cast<uint8_t>(TypeIndex::UInt8)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("UInt8")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"),
+              static_cast<uint8_t>(TypeIndex::UInt8)},
              Null()}, // complex array
     };
-
     for (const auto& row : data_set) {
         DataSet const_dataset = {row};
         check_function<DataTypeUInt8, true>(func_name, input_types, const_dataset);
     }
-
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::Int8}};
     // cast to TINYINT
     data_set = {
-            {{STRING("null"), STRING("Int8")}, Null()},
-            {{STRING("true"), STRING("Int8")}, Null()},
-            {{STRING("false"), STRING("Int8")}, Null()},
-            {{STRING("100"), STRING("Int8")}, TINYINT(100)},                  //int8
-            {{STRING("10000"), STRING("Int8")}, Null()},                      // int16
-            {{STRING("1000000000"), STRING("Int8")}, Null()},                 // int32
-            {{STRING("1152921504606846976"), STRING("Int8")}, Null()},        // int64
-            {{STRING("6.18"), STRING("Int8")}, Null()},                       // double
-            {{STRING(R"("abcd")"), STRING("Int8")}, Null()},                  // string
-            {{STRING("{}"), STRING("Int8")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("Int8")}, Null()}, // object
-            {{STRING("[]"), STRING("Int8")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("Int8")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("Int8")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("Int8")},
+            {{STRING("null"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},
+            {{STRING("true"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},
+            {{STRING("false"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},
+            {{STRING("100"), static_cast<int8_t>(TypeIndex::Int8)}, TINYINT(100)},  //int8
+            {{STRING("10000"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},      // int16
+            {{STRING("1000000000"), static_cast<int8_t>(TypeIndex::Int8)}, Null()}, // int32
+            {{STRING("1152921504606846976"), static_cast<int8_t>(TypeIndex::Int8)},
+             Null()},                                                              // int64
+            {{STRING("6.18"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},      // double
+            {{STRING(R"("abcd")"), static_cast<int8_t>(TypeIndex::Int8)}, Null()}, // string
+            {{STRING("{}"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},        // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), static_cast<int8_t>(TypeIndex::Int8)},
+             Null()},                                                               // object
+            {{STRING("[]"), static_cast<int8_t>(TypeIndex::Int8)}, Null()},         // empty array
+            {{STRING("[123, 456]"), static_cast<int8_t>(TypeIndex::Int8)}, Null()}, // int array
+            {{STRING(R"(["abc", "def"])"), static_cast<int8_t>(TypeIndex::Int8)},
+             Null()}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"),
+              static_cast<int8_t>(TypeIndex::Int8)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int8")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"),
+              static_cast<int8_t>(TypeIndex::Int8)},
              Null()}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1257,25 +1266,31 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
         check_function<DataTypeInt8, true>(func_name, input_types, const_dataset);
     }
 
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::Int16}};
     // cast to SMALLINT
     data_set = {
-            {{STRING("null"), STRING("Int16")}, Null()},
-            {{STRING("true"), STRING("Int16")}, Null()},
-            {{STRING("false"), STRING("Int16")}, Null()},
-            {{STRING("100"), STRING("Int16")}, SMALLINT(100)},                 //int8
-            {{STRING("10000"), STRING("Int16")}, SMALLINT(10000)},             // int16
-            {{STRING("1000000000"), STRING("Int16")}, Null()},                 // int32
-            {{STRING("1152921504606846976"), STRING("Int16")}, Null()},        // int64
-            {{STRING("6.18"), STRING("Int16")}, Null()},                       // double
-            {{STRING(R"("abcd")"), STRING("Int16")}, Null()},                  // string
-            {{STRING("{}"), STRING("Int16")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("Int16")}, Null()}, // object
-            {{STRING("[]"), STRING("Int16")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("Int16")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("Int16")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("Int16")},
+            {{STRING("null"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},
+            {{STRING("true"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},
+            {{STRING("false"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},
+            {{STRING("100"), static_cast<int16_t>(TypeIndex::Int16)}, SMALLINT(100)},     //int8
+            {{STRING("10000"), static_cast<int16_t>(TypeIndex::Int16)}, SMALLINT(10000)}, // int16
+            {{STRING("1000000000"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},     // int32
+            {{STRING("1152921504606846976"), static_cast<int16_t>(TypeIndex::Int16)},
+             Null()},                                                                // int64
+            {{STRING("6.18"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},      // double
+            {{STRING(R"("abcd")"), static_cast<int16_t>(TypeIndex::Int16)}, Null()}, // string
+            {{STRING("{}"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},        // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), static_cast<int16_t>(TypeIndex::Int16)},
+             Null()},                                                                 // object
+            {{STRING("[]"), static_cast<int16_t>(TypeIndex::Int16)}, Null()},         // empty array
+            {{STRING("[123, 456]"), static_cast<int16_t>(TypeIndex::Int16)}, Null()}, // int array
+            {{STRING(R"(["abc", "def"])"), static_cast<int16_t>(TypeIndex::Int16)},
+             Null()}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"),
+              static_cast<int16_t>(TypeIndex::Int16)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int16")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"),
+              static_cast<int16_t>(TypeIndex::Int16)},
              Null()}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1283,25 +1298,32 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
         check_function<DataTypeInt16, true>(func_name, input_types, const_dataset);
     }
 
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::Int32}};
     // cast to INT
     data_set = {
-            {{STRING("null"), STRING("Int32")}, Null()},
-            {{STRING("true"), STRING("Int32")}, Null()},
-            {{STRING("false"), STRING("Int32")}, Null()},
-            {{STRING("100"), STRING("Int32")}, INT(100)},                      //int8
-            {{STRING("10000"), STRING("Int32")}, INT(10000)},                  // int16
-            {{STRING("1000000000"), STRING("Int32")}, INT(1000000000)},        // int32
-            {{STRING("1152921504606846976"), STRING("Int32")}, Null()},        // int64
-            {{STRING("6.18"), STRING("Int32")}, Null()},                       // double
-            {{STRING(R"("abcd")"), STRING("Int32")}, Null()},                  // string
-            {{STRING("{}"), STRING("Int32")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("Int32")}, Null()}, // object
-            {{STRING("[]"), STRING("Int32")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("Int32")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("Int32")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("Int32")},
+            {{STRING("null"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},
+            {{STRING("true"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},
+            {{STRING("false"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},
+            {{STRING("100"), static_cast<int32_t>(TypeIndex::Int32)}, INT(100)},     //int8
+            {{STRING("10000"), static_cast<int32_t>(TypeIndex::Int32)}, INT(10000)}, // int16
+            {{STRING("1000000000"), static_cast<int32_t>(TypeIndex::Int32)},
+             INT(1000000000)}, // int32
+            {{STRING("1152921504606846976"), static_cast<int32_t>(TypeIndex::Int32)},
+             Null()},                                                                // int64
+            {{STRING("6.18"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},      // double
+            {{STRING(R"("abcd")"), static_cast<int32_t>(TypeIndex::Int32)}, Null()}, // string
+            {{STRING("{}"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},        // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), static_cast<int32_t>(TypeIndex::Int32)},
+             Null()},                                                                 // object
+            {{STRING("[]"), static_cast<int32_t>(TypeIndex::Int32)}, Null()},         // empty array
+            {{STRING("[123, 456]"), static_cast<int32_t>(TypeIndex::Int32)}, Null()}, // int array
+            {{STRING(R"(["abc", "def"])"), static_cast<int32_t>(TypeIndex::Int32)},
+             Null()}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"),
+              static_cast<int32_t>(TypeIndex::Int32)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int32")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"),
+              static_cast<int32_t>(TypeIndex::Int32)},
              Null()}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1309,26 +1331,26 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
         check_function<DataTypeInt32, true>(func_name, input_types, const_dataset);
     }
 
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::Int64}};
     // cast to BIGINT
     data_set = {
-            {{STRING("null"), STRING("Int64")}, Null()},
-            {{STRING("true"), STRING("Int64")}, Null()},
-            {{STRING("false"), STRING("Int64")}, Null()},
-            {{STRING("100"), STRING("Int64")}, BIGINT(100)},               //int8
-            {{STRING("10000"), STRING("Int64")}, BIGINT(10000)},           // int16
-            {{STRING("1000000000"), STRING("Int64")}, BIGINT(1000000000)}, // int32
-            {{STRING("1152921504606846976"), STRING("Int64")},
-             BIGINT(1152921504606846976)},                                     // int64
-            {{STRING("6.18"), STRING("Int64")}, Null()},                       // double
-            {{STRING(R"("abcd")"), STRING("Int64")}, Null()},                  // string
-            {{STRING("{}"), STRING("Int64")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("Int64")}, Null()}, // object
-            {{STRING("[]"), STRING("Int64")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("Int64")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("Int64")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("Int64")},
+            {{STRING("null"), BIGINT(1)}, Null()},
+            {{STRING("true"), BIGINT(1)}, Null()},
+            {{STRING("false"), BIGINT(1)}, Null()},
+            {{STRING("100"), BIGINT(1)}, BIGINT(100)},                                 //int8
+            {{STRING("10000"), BIGINT(1)}, BIGINT(10000)},                             // int16
+            {{STRING("1000000000"), BIGINT(1)}, BIGINT(1000000000)},                   // int32
+            {{STRING("1152921504606846976"), BIGINT(1)}, BIGINT(1152921504606846976)}, // int64
+            {{STRING("6.18"), BIGINT(1)}, Null()},                                     // double
+            {{STRING(R"("abcd")"), BIGINT(1)}, Null()},                                // string
+            {{STRING("{}"), BIGINT(1)}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), BIGINT(1)}, Null()}, // object
+            {{STRING("[]"), BIGINT(1)}, Null()},                         // empty array
+            {{STRING("[123, 456]"), BIGINT(1)}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), BIGINT(1)}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), BIGINT(1)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int64")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), BIGINT(1)},
              Null()}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1336,26 +1358,26 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
         check_function<DataTypeInt64, true>(func_name, input_types, const_dataset);
     }
 
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::Float64}};
     // cast to DOUBLE
     data_set = {
-            {{STRING("null"), STRING("Float64")}, Null()},
-            {{STRING("true"), STRING("Float64")}, Null()},
-            {{STRING("false"), STRING("Float64")}, Null()},
-            {{STRING("100"), STRING("Float64")}, DOUBLE(100)},               //int8
-            {{STRING("10000"), STRING("Float64")}, DOUBLE(10000)},           // int16
-            {{STRING("1000000000"), STRING("Float64")}, DOUBLE(1000000000)}, // int32
-            {{STRING("1152921504606846976"), STRING("Float64")},
-             DOUBLE(1152921504606846976)},                                       // int64
-            {{STRING("6.18"), STRING("Float64")}, DOUBLE(6.18)},                 // double
-            {{STRING(R"("abcd")"), STRING("Float64")}, Null()},                  // string
-            {{STRING("{}"), STRING("Float64")}, Null()},                         // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("Float64")}, Null()}, // object
-            {{STRING("[]"), STRING("Float64")}, Null()},                         // empty array
-            {{STRING("[123, 456]"), STRING("Float64")}, Null()},                 // int array
-            {{STRING(R"(["abc", "def"])"), STRING("Float64")}, Null()},          // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("Float64")},
+            {{STRING("null"), DOUBLE(1)}, Null()},
+            {{STRING("true"), DOUBLE(1)}, Null()},
+            {{STRING("false"), DOUBLE(1)}, Null()},
+            {{STRING("100"), DOUBLE(1)}, DOUBLE(100)},                                 //int8
+            {{STRING("10000"), DOUBLE(1)}, DOUBLE(10000)},                             // int16
+            {{STRING("1000000000"), DOUBLE(1)}, DOUBLE(1000000000)},                   // int32
+            {{STRING("1152921504606846976"), DOUBLE(1)}, DOUBLE(1152921504606846976)}, // int64
+            {{STRING("6.18"), DOUBLE(1)}, DOUBLE(6.18)},                               // double
+            {{STRING(R"("abcd")"), DOUBLE(1)}, Null()},                                // string
+            {{STRING("{}"), DOUBLE(1)}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), DOUBLE(1)}, Null()}, // object
+            {{STRING("[]"), DOUBLE(1)}, Null()},                         // empty array
+            {{STRING("[123, 456]"), DOUBLE(1)}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), DOUBLE(1)}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), DOUBLE(1)},
              Null()}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Float64")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), DOUBLE(1)},
              Null()}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1363,28 +1385,28 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
         check_function<DataTypeFloat64, true>(func_name, input_types, const_dataset);
     }
 
+    input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::String}};
     // cast to STRING
     data_set = {
-            {{STRING("null"), STRING("String")}, STRING("null")},
-            {{STRING("true"), STRING("String")}, STRING("true")},
-            {{STRING("false"), STRING("String")}, STRING("false")},
-            {{STRING("100"), STRING("String")}, STRING("100")},               //int8
-            {{STRING("10000"), STRING("String")}, STRING("10000")},           // int16
-            {{STRING("1000000000"), STRING("String")}, STRING("1000000000")}, // int32
-            {{STRING("1152921504606846976"), STRING("String")},
-             STRING("1152921504606846976")},                                // int64
-            {{STRING("6.18"), STRING("String")}, STRING("6.18")},           // double
-            {{STRING(R"("abcd")"), STRING("String")}, STRING(R"("abcd")")}, // string
-            {{STRING("{}"), STRING("String")}, STRING("{}")},               // empty object
-            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("String")},
-             STRING(R"({"k1":"v31","k2":300})")},                            // object
-            {{STRING("[]"), STRING("String")}, STRING("[]")},                // empty array
-            {{STRING("[123, 456]"), STRING("String")}, STRING("[123,456]")}, // int array
-            {{STRING(R"(["abc", "def"])"), STRING("String")},
+            {{STRING("null"), STRING("1")}, STRING("null")},
+            {{STRING("true"), STRING("1")}, STRING("true")},
+            {{STRING("false"), STRING("1")}, STRING("false")},
+            {{STRING("100"), STRING("1")}, STRING("100")},                                 //int8
+            {{STRING("10000"), STRING("1")}, STRING("10000")},                             // int16
+            {{STRING("1000000000"), STRING("1")}, STRING("1000000000")},                   // int32
+            {{STRING("1152921504606846976"), STRING("1")}, STRING("1152921504606846976")}, // int64
+            {{STRING("6.18"), STRING("1")}, STRING("6.18")},                               // double
+            {{STRING(R"("abcd")"), STRING("1")}, STRING(R"("abcd")")},                     // string
+            {{STRING("{}"), STRING("1")}, STRING("{}")}, // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("1")},
+             STRING(R"({"k1":"v31","k2":300})")},                       // object
+            {{STRING("[]"), STRING("1")}, STRING("[]")},                // empty array
+            {{STRING("[123, 456]"), STRING("1")}, STRING("[123,456]")}, // int array
+            {{STRING(R"(["abc", "def"])"), STRING("1")},
              STRING(R"(["abc","def"])")}, // string array
-            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("String")},
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("1")},
              STRING(R"([null,true,false,100,6.18,"abc"])")}, // multi type array
-            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("String")},
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("1")},
              STRING(R"([{"k1":"v41","k2":400},1,"a",3.14])")}, // complex array
     };
     for (const auto& row : data_set) {
@@ -1396,62 +1418,61 @@ TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
 TEST(FunctionJsonbTEST, JsonbCastFromOtherTest) {
     // CAST Nullable(X) to Nullable(JSONB)
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
-            {{{BOOLEAN(1), STRING("Jsonb")}, STRING("true")}});
+            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BOOLEAN(1), Null()}, STRING("true")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
-            {{{BOOLEAN(0), STRING("Jsonb")}, STRING("false")}});
+            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BOOLEAN(0), Null()}, STRING("false")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::Int8}, ConstedNotnull {TypeIndex::String}},
-            {{{TINYINT(100), STRING("Jsonb")}, STRING("100")}});
+            "CAST", {Nullable {TypeIndex::Int8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{TINYINT(100), Null()}, STRING("100")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::Int16}, ConstedNotnull {TypeIndex::String}},
-            {{{SMALLINT(10000), STRING("Jsonb")}, STRING("10000")}});
+            "CAST", {Nullable {TypeIndex::Int16}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{SMALLINT(10000), Null()}, STRING("10000")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::Int32}, ConstedNotnull {TypeIndex::String}},
-            {{{INT(1000000000), STRING("Jsonb")}, STRING("1000000000")}});
+            "CAST", {Nullable {TypeIndex::Int32}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{INT(1000000000), Null()}, STRING("1000000000")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::Int64}, ConstedNotnull {TypeIndex::String}},
-            {{{BIGINT(1152921504606846976), STRING("Jsonb")}, STRING("1152921504606846976")}});
+            "CAST", {Nullable {TypeIndex::Int64}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BIGINT(1152921504606846976), Null()}, STRING("1152921504606846976")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::Float64}, ConstedNotnull {TypeIndex::String}},
-            {{{DOUBLE(6.18), STRING("Jsonb")}, STRING("6.18")}});
+            "CAST", {Nullable {TypeIndex::Float64}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{DOUBLE(6.18), Null()}, STRING("6.18")}});
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
-            {{{STRING(R"(abcd)"), STRING("Jsonb")}, Null()}}); // should fail
+            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{STRING(R"(abcd)"), Null()}, Null()}}); // should fail
     check_function<DataTypeJsonb, true>(
-            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
-            {{{STRING(R"("abcd")"), STRING("Jsonb")}, STRING(R"("abcd")")}});
+            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{STRING(R"("abcd")"), Null()}, STRING(R"("abcd")")}});
 
     // CAST X to JSONB
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
-            {{{BOOLEAN(1), STRING("Jsonb")}, STRING("true")}});
+            "CAST", {Notnull {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BOOLEAN(1), Null()}, STRING("true")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
-            {{{BOOLEAN(0), STRING("Jsonb")}, STRING("false")}});
+            "CAST", {Notnull {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BOOLEAN(0), Null()}, STRING("false")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::Int8}, ConstedNotnull {TypeIndex::String}},
-            {{{TINYINT(100), STRING("Jsonb")}, STRING("100")}});
+            "CAST", {Notnull {TypeIndex::Int8}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{TINYINT(100), Null()}, STRING("100")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::Int16}, ConstedNotnull {TypeIndex::String}},
-            {{{SMALLINT(10000), STRING("Jsonb")}, STRING("10000")}});
+            "CAST", {Notnull {TypeIndex::Int16}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{SMALLINT(10000), Null()}, STRING("10000")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::Int32}, ConstedNotnull {TypeIndex::String}},
-            {{{INT(1000000000), STRING("Jsonb")}, STRING("1000000000")}});
+            "CAST", {Notnull {TypeIndex::Int32}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{INT(1000000000), Null()}, STRING("1000000000")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::Int64}, ConstedNotnull {TypeIndex::String}},
-            {{{BIGINT(1152921504606846976), STRING("Jsonb")}, STRING("1152921504606846976")}});
+            "CAST", {Notnull {TypeIndex::Int64}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{BIGINT(1152921504606846976), Null()}, STRING("1152921504606846976")}});
     check_function<DataTypeJsonb, false>(
-            "CAST", {Notnull {TypeIndex::Float64}, ConstedNotnull {TypeIndex::String}},
-            {{{DOUBLE(6.18), STRING("Jsonb")}, STRING("6.18")}});
+            "CAST", {Notnull {TypeIndex::Float64}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{DOUBLE(6.18), Null()}, STRING("6.18")}});
     // String to JSONB should always be Nullable
     check_function<DataTypeJsonb, true>(
-            "CAST", {Notnull {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
-            {{{STRING(R"(abcd)"), STRING("Jsonb")}, Null()}}); // should fail
+            "CAST", {Notnull {TypeIndex::String}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{STRING(R"(abcd)"), Null()}, Null()}}); // should fail
     check_function<DataTypeJsonb, true>(
-            "CAST", {Notnull {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
-            {{{STRING(R"("abcd")"), STRING("Jsonb")}, STRING(R"("abcd")")}});
+            "CAST", {Notnull {TypeIndex::String}, ConstedNotnull {TypeIndex::JSONB}},
+            {{{STRING(R"("abcd")"), Null()}, STRING(R"("abcd")")}});
 }
-
 } // namespace doris::vectorized


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