You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/10/21 07:18:22 UTC

[doris] branch master updated: [feature](jsonb type) functions for cast from and to jsonb datatype (#13379)

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

morningman 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 ccc04210d6 [feature](jsonb type) functions for cast from and to jsonb datatype (#13379)
ccc04210d6 is described below

commit ccc04210d694ab5be8341942ef57afc56ae9f3b7
Author: Kang <kx...@gmail.com>
AuthorDate: Fri Oct 21 15:18:16 2022 +0800

    [feature](jsonb type) functions for cast from and to jsonb datatype (#13379)
---
 be/src/olap/types.h                          |  37 +--
 be/src/runtime/jsonb_value.cpp               |  13 +-
 be/src/runtime/jsonb_value.h                 |   2 +-
 be/src/util/jsonb_utils.h                    |  15 +-
 be/src/vec/data_types/data_type_jsonb.cpp    |  23 +-
 be/src/vec/data_types/data_type_jsonb.h      |   1 +
 be/src/vec/functions/function_cast.h         | 286 ++++++++++++++++++-
 be/src/vec/sink/vmysql_result_writer.cpp     |   6 +-
 be/test/vec/core/column_jsonb_test.cpp       |   4 +-
 be/test/vec/function/function_jsonb_test.cpp | 403 ++++++++++++++++++++-------
 be/test/vec/function/function_test_util.cpp  |  19 +-
 be/test/vec/function/function_test_util.h    |  25 +-
 12 files changed, 651 insertions(+), 183 deletions(-)

diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index f8e98943bc..8754e610c6 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -1576,43 +1576,14 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_JSONB> : public FieldTypeTraits<OLAP_FIEL
 
     static Status from_string(void* buf, const std::string& scan_key, const int precision,
                               const int scale) {
-        JsonBinaryValue binary_val(scan_key.c_str(), scan_key.size());
-        auto jdoc = JsonbDocument::createDocument(binary_val.value(), binary_val.size());
-        size_t value_len = jdoc->numPackedBytes();
-        if (value_len > config::jsonb_type_length_soft_limit_bytes) {
-            LOG(WARNING) << "the len of value json is too long, len=" << value_len
-                         << ", max_len=" << config::jsonb_type_length_soft_limit_bytes;
-            return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
-        }
-
-        auto slice = reinterpret_cast<Slice*>(buf);
-        memory_copy(slice->data, reinterpret_cast<const char*>(jdoc->getValue()), value_len);
-        slice->size = value_len;
-        return Status::OK();
+        // TODO support schema change
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
     }
 
     static Status convert_from(void* dest, const void* src, const TypeInfo* src_type,
                                MemPool* mem_pool, size_t variable_len = 0) {
-        JsonbToJson toStr;
-        switch (src_type->type()) {
-        // TODO(wzy): JSONB should support all numerics
-        case OLAP_FIELD_TYPE_CHAR:
-        case OLAP_FIELD_TYPE_VARCHAR:
-        case OLAP_FIELD_TYPE_STRING: {
-            auto s = src_type->to_string(src);
-            JsonBinaryValue binary_val(s.c_str(), s.size());
-            std::string result = toStr.jsonb_to_string(
-                    JsonbDocument::createDocument(binary_val.value(), binary_val.size())
-                            ->getValue());
-            auto slice = reinterpret_cast<Slice*>(dest);
-            slice->data = reinterpret_cast<char*>(mem_pool->allocate(result.size()));
-            memcpy(slice->data, result.c_str(), result.size());
-            slice->size = result.size();
-            return Status::OK();
-        }
-        default:
-            return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
-        }
+        // TODO support schema change
+        return Status::OLAPInternalError(OLAP_ERR_INVALID_SCHEMA);
     }
 
     static void set_to_min(void* buf) {
diff --git a/be/src/runtime/jsonb_value.cpp b/be/src/runtime/jsonb_value.cpp
index be59e1fd96..93abcce640 100644
--- a/be/src/runtime/jsonb_value.cpp
+++ b/be/src/runtime/jsonb_value.cpp
@@ -23,23 +23,22 @@
 
 namespace doris {
 
-JsonbErrType JsonBinaryValue::from_json_string(const char* s, int length) {
+Status JsonBinaryValue::from_json_string(const char* s, int length) {
     JsonbErrType error = JsonbErrType::E_NONE;
     if (!parser.parse(s, length)) {
         error = parser.getErrorCode();
-        ptr = nullptr;
-        len = 0;
-        return error;
+        return Status::InvalidArgument("json parse error: {} for value: {}",
+                                       JsonbErrMsg::getErrMsg(error), std::string_view(s, length));
     }
+
     ptr = parser.getWriter().getOutput()->getBuffer();
     len = (unsigned)parser.getWriter().getOutput()->getSize();
     DCHECK_LE(len, MAX_LENGTH);
-    return error;
+    return Status::OK();
 }
 
 std::string JsonBinaryValue::to_json_string() const {
-    JsonbToJson toStr;
-    return toStr.jsonb_to_string(JsonbDocument::createDocument(ptr, len)->getValue());
+    return JsonbToJson::jsonb_to_json_string(ptr, len);
 }
 
 std::ostream& operator<<(std::ostream& os, const JsonBinaryValue& json_value) {
diff --git a/be/src/runtime/jsonb_value.h b/be/src/runtime/jsonb_value.h
index 4278070b83..5e99979a5c 100644
--- a/be/src/runtime/jsonb_value.h
+++ b/be/src/runtime/jsonb_value.h
@@ -97,7 +97,7 @@ struct JsonBinaryValue {
         LOG(FATAL) << "comparing between JsonBinaryValue is not supported";
     }
 
-    JsonbErrType from_json_string(const char* s, int len);
+    Status from_json_string(const char* s, int len);
 
     std::string to_json_string() const;
 
diff --git a/be/src/util/jsonb_utils.h b/be/src/util/jsonb_utils.h
index ccfc431330..64e125e329 100644
--- a/be/src/util/jsonb_utils.h
+++ b/be/src/util/jsonb_utils.h
@@ -39,7 +39,9 @@ public:
     JsonbToJson() : os_(buffer_, OUT_BUF_SIZE) {}
 
     // get json string
-    const char* jsonb_to_string(const JsonbValue* pval) {
+    const std::string to_json_string(const char* data, size_t size) {
+        doris::JsonbValue* pval = doris::JsonbDocument::createDocument(data, size)->getValue();
+
         os_.clear();
         os_.seekp(0);
 
@@ -48,13 +50,14 @@ public:
         }
 
         os_.put(0);
-        return os_.getBuffer();
+
+        std::string json_string {os_.getBuffer()};
+        return json_string;
     }
 
-    const char* jsonb_to_string(const char* data, size_t size) {
-        doris::JsonbToJson toStr;
-        doris::JsonbValue* val = doris::JsonbDocument::createDocument(data, size)->getValue();
-        return toStr.jsonb_to_string(val);
+    static const std::string jsonb_to_json_string(const char* data, size_t size) {
+        JsonbToJson jsonb_to_json;
+        return jsonb_to_json.to_json_string(data, size);
     }
 
 private:
diff --git a/be/src/vec/data_types/data_type_jsonb.cpp b/be/src/vec/data_types/data_type_jsonb.cpp
index 1e0c872f7c..c607ced9b3 100644
--- a/be/src/vec/data_types/data_type_jsonb.cpp
+++ b/be/src/vec/data_types/data_type_jsonb.cpp
@@ -49,21 +49,28 @@ static inline void read(IColumn& column, Reader&& reader) {
 }
 
 std::string DataTypeJsonb::to_string(const IColumn& column, size_t row_num) const {
-    JsonbToJson toStr;
     const StringRef& s =
             reinterpret_cast<const ColumnJsonb&>(*column.convert_to_full_column_if_const().get())
                     .get_data_at(row_num);
-    return toStr.jsonb_to_string(s.data, s.size);
+    return JsonbToJson::jsonb_to_json_string(s.data, s.size);
 }
 
 void DataTypeJsonb::to_string(const class doris::vectorized::IColumn& column, size_t row_num,
                               class doris::vectorized::BufferWritable& ostr) const {
-    JsonbToJson toStr;
-    const StringRef& s =
-            reinterpret_cast<const ColumnJsonb&>(*column.convert_to_full_column_if_const().get())
-                    .get_data_at(row_num);
-    std::string str = toStr.jsonb_to_string(s.data, s.size);
-    ostr.write(str.c_str(), str.size());
+    std::string json_string = to_string(column, row_num);
+    ostr.write(json_string.c_str(), json_string.size());
+}
+
+Status DataTypeJsonb::from_string(ReadBuffer& rb, IColumn* column) const {
+    JsonBinaryValue value;
+    RETURN_IF_ERROR(value.from_json_string(rb.position(), rb.count()));
+
+    Field field = JsonbField(value.value(), value.size());
+
+    auto* column_jsonb = static_cast<ColumnJsonb*>(column);
+    column_jsonb->insert(field);
+
+    return Status::OK();
 }
 
 MutableColumnPtr DataTypeJsonb::create_column() const {
diff --git a/be/src/vec/data_types/data_type_jsonb.h b/be/src/vec/data_types/data_type_jsonb.h
index 7d9fd45a69..f437f40c3a 100644
--- a/be/src/vec/data_types/data_type_jsonb.h
+++ b/be/src/vec/data_types/data_type_jsonb.h
@@ -59,5 +59,6 @@ public:
     bool can_be_inside_low_cardinality() const override { return true; }
     std::string to_string(const IColumn& column, size_t row_num) const override;
     void to_string(const IColumn& column, size_t row_num, BufferWritable& ostr) const override;
+    Status from_string(ReadBuffer& rb, IColumn* column) const override;
 };
 } // namespace doris::vectorized
\ No newline at end of file
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index 396b676103..befb22532a 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -29,9 +29,11 @@
 #include "vec/columns/column_string.h"
 #include "vec/columns/columns_common.h"
 #include "vec/common/assert_cast.h"
+#include "vec/common/field_visitors.h"
 #include "vec/common/string_buffer.hpp"
 #include "vec/data_types/data_type_decimal.h"
 #include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/data_type_jsonb.h"
 #include "vec/data_types/data_type_nullable.h"
 #include "vec/data_types/data_type_number.h"
 #include "vec/data_types/data_type_string.h"
@@ -303,6 +305,11 @@ struct ConvertImplGenericToString {
         block.replace_by_position(result, std::move(col_to));
         return Status::OK();
     }
+
+    static Status execute2(FunctionContext* /*ctx*/, Block& block, const ColumnNumbers& arguments,
+                           const size_t result, size_t /*input_rows_count*/) {
+        return execute(block, arguments, result);
+    }
 };
 
 template <typename StringColumnType>
@@ -352,6 +359,188 @@ struct ConvertImplGenericFromString {
     }
 };
 
+// Generic conversion of number to jsonb.
+template <typename ColumnType>
+struct ConvertImplNumberToJsonb {
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          const size_t result, size_t input_rows_count) {
+        const auto& col_with_type_and_name = block.get_by_position(arguments[0]);
+
+        auto col_jsonb = ColumnJsonb::create();
+        JsonbWriter writer;
+
+        const auto* col =
+                check_and_get_column<const ColumnType>(col_with_type_and_name.column.get());
+        const auto& data = col->get_data();
+
+        for (size_t i = 0; i < input_rows_count; i++) {
+            writer.reset();
+            if constexpr (std::is_same_v<ColumnUInt8, ColumnType>) {
+                writer.writeBool(data[i]);
+            } else if constexpr (std::is_same_v<ColumnInt8, ColumnType>) {
+                writer.writeInt8(data[i]);
+            } else if constexpr (std::is_same_v<ColumnInt16, ColumnType>) {
+                writer.writeInt16(data[i]);
+            } else if constexpr (std::is_same_v<ColumnInt32, ColumnType>) {
+                writer.writeInt32(data[i]);
+            } else if constexpr (std::is_same_v<ColumnInt64, ColumnType>) {
+                writer.writeInt64(data[i]);
+            } else if constexpr (std::is_same_v<ColumnFloat64, ColumnType>) {
+                writer.writeDouble(data[i]);
+            } else {
+                LOG(FATAL) << "unsupported type ";
+            }
+            col_jsonb->insert_data(writer.getOutput()->getBuffer(), writer.getOutput()->getSize());
+        }
+
+        block.replace_by_position(result, std::move(col_jsonb));
+        return Status::OK();
+    }
+};
+
+// Generic conversion of any type to jsonb.
+struct ConvertImplGenericToJsonb {
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          const size_t result, size_t input_rows_count) {
+        const auto& col_with_type_and_name = block.get_by_position(arguments[0]);
+        const IDataType& type = *col_with_type_and_name.type;
+        const IColumn& col_from = *col_with_type_and_name.column;
+
+        auto col_jsonb = ColumnJsonb::create();
+        JsonbWriter writer;
+
+        auto tmp_col = ColumnString::create();
+        for (size_t i = 0; i < input_rows_count; i++) {
+            // convert to string
+            tmp_col->clear();
+            VectorBufferWriter write_buffer(*tmp_col.get());
+            type.to_string(col_from, i, write_buffer);
+            write_buffer.commit();
+            // write string to jsonb
+            writer.reset();
+            writer.writeStartString();
+            auto str_ref = tmp_col->get_data_at(0);
+            writer.writeString(str_ref.data, str_ref.size);
+            writer.writeEndString();
+            col_jsonb->insert_data(writer.getOutput()->getBuffer(), writer.getOutput()->getSize());
+        }
+
+        block.replace_by_position(result, std::move(col_jsonb));
+        return Status::OK();
+    }
+};
+
+template <TypeIndex type_index, typename ColumnType>
+struct ConvertImplFromJsonb {
+    static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                          const size_t result, size_t input_rows_count) {
+        const auto& col_with_type_and_name = block.get_by_position(arguments[0]);
+        const IColumn& col_from = *col_with_type_and_name.column;
+        // result column must set type
+        DCHECK(block.get_by_position(result).type != nullptr);
+        auto data_type_to = block.get_by_position(result).type;
+        if (const ColumnJsonb* col_jsonb = check_and_get_column<ColumnJsonb>(&col_from)) {
+            auto null_map_col = ColumnUInt8::create(input_rows_count, 0);
+            auto& null_map = null_map_col->get_data();
+            auto col_to = ColumnType::create();
+
+            //IColumn & col_to = *res;
+            // size_t size = col_from.size();
+            col_to->reserve(input_rows_count);
+            auto& res = col_to->get_data();
+            res.resize(input_rows_count);
+
+            for (size_t i = 0; i < input_rows_count; ++i) {
+                const auto& val = col_jsonb->get_data_at(i);
+                // ReadBuffer read_buffer((char*)(val.data), val.size);
+                // RETURN_IF_ERROR(data_type_to->from_string(read_buffer, col_to));
+
+                if (val.size == 0) {
+                    null_map[i] = 1;
+                    res[i] = 0;
+                    continue;
+                }
+
+                // doc is NOT necessary to be deleted since JsonbDocument will not allocate memory
+                JsonbDocument* doc = JsonbDocument::createDocument(val.data, val.size);
+                if (UNLIKELY(!doc || !doc->getValue())) {
+                    null_map[i] = 1;
+                    res[i] = 0;
+                    continue;
+                }
+
+                // value is NOT necessary to be deleted since JsonbValue will not allocate memory
+                JsonbValue* value = doc->getValue();
+                if (UNLIKELY(!value)) {
+                    null_map[i] = 1;
+                    res[i] = 0;
+                    continue;
+                }
+
+                if constexpr (type_index == TypeIndex::UInt8) {
+                    if (value->isTrue()) {
+                        res[i] = 1;
+                    } else if (value->isFalse()) {
+                        res[i] = 0;
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else if constexpr (type_index == TypeIndex::Int8) {
+                    if (value->isInt8()) {
+                        res[i] = ((const JsonbIntVal*)value)->val();
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else if constexpr (type_index == TypeIndex::Int16) {
+                    if (value->isInt8() || value->isInt16()) {
+                        res[i] = (int16_t)((const JsonbIntVal*)value)->val();
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else if constexpr (type_index == TypeIndex::Int32) {
+                    if (value->isInt8() || value->isInt16() || value->isInt32()) {
+                        res[i] = (int32_t)((const JsonbIntVal*)value)->val();
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else if constexpr (type_index == TypeIndex::Int64) {
+                    if (value->isInt8() || value->isInt16() || value->isInt32() ||
+                        value->isInt64()) {
+                        res[i] = ((const JsonbIntVal*)value)->val();
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else if constexpr (type_index == TypeIndex::Float64) {
+                    if (value->isDouble()) {
+                        res[i] = ((const JsonbDoubleVal*)value)->val();
+                    } else if (value->isInt8() || value->isInt16() || value->isInt32() ||
+                               value->isInt64()) {
+                        res[i] = ((const JsonbIntVal*)value)->val();
+                    } else {
+                        null_map[i] = 1;
+                        res[i] = 0;
+                    }
+                } else {
+                    LOG(FATAL) << "unsupported type ";
+                }
+            }
+
+            block.replace_by_position(
+                    result, ColumnNullable::create(std::move(col_to), std::move(null_map_col)));
+        } else {
+            return Status::RuntimeError(
+                    "Illegal column {} of first argument of conversion function from string",
+                    col_from.get_name());
+        }
+        return Status::OK();
+    }
+};
+
 template <typename ToDataType, typename Name>
 struct ConvertImpl<DataTypeString, ToDataType, Name> {
     template <typename Additions = void*>
@@ -1132,9 +1321,7 @@ private:
                   which.is_float() || which.is_date_or_datetime() ||
                   which.is_date_v2_or_datetime_v2() || which.is_string_or_fixed_string();
         if (!ok) {
-            LOG(FATAL) << fmt::format(
-                    "Conversion from {} to {} to_type->get_name() is not supported",
-                    from_type->get_name(), to_type->get_name());
+            return create_unsupport_wrapper(from_type->get_name(), to_type->get_name());
         }
 
         return [type_index, precision, scale](FunctionContext* context, Block& block,
@@ -1180,6 +1367,22 @@ private:
         };
     }
 
+    WrapperType create_unsupport_wrapper(const String error_msg) const {
+        LOG(WARNING) << error_msg;
+        return [error_msg](FunctionContext* /*context*/, Block& /*block*/,
+                           const ColumnNumbers& /*arguments*/, const size_t /*result*/,
+                           size_t /*input_rows_count*/) {
+            return Status::InvalidArgument(error_msg);
+        };
+    }
+
+    WrapperType create_unsupport_wrapper(const String from_type_name,
+                                         const String to_type_name) const {
+        const String error_msg = fmt::format("Conversion from {} to {} is not supported",
+                                             from_type_name, to_type_name);
+        return create_unsupport_wrapper(error_msg);
+    }
+
     WrapperType create_array_wrapper(const DataTypePtr& from_type_untyped,
                                      const DataTypeArray& to_type) const {
         /// Conversion from String through parsing.
@@ -1190,8 +1393,9 @@ private:
         const auto* from_type = check_and_get_data_type<DataTypeArray>(from_type_untyped.get());
 
         if (!from_type) {
-            LOG(FATAL) << "CAST AS Array can only be performed between same-dimensional Array, "
-                          "String types";
+            return create_unsupport_wrapper(
+                    "CAST AS Array can only be performed between same-dimensional Array, String "
+                    "types");
         }
 
         DataTypePtr from_nested_type = from_type->get_nested_type();
@@ -1201,8 +1405,8 @@ private:
 
         if (from_type->get_number_of_dimensions() != to_type.get_number_of_dimensions() &&
             !from_empty_array) {
-            LOG(FATAL)
-                    << "CAST AS Array can only be performed between same-dimensional array types";
+            return create_unsupport_wrapper(
+                    "CAST AS Array can only be performed between same-dimensional array types");
         }
 
         const DataTypePtr& to_nested_type = to_type.get_nested_type();
@@ -1244,6 +1448,58 @@ private:
         };
     }
 
+    // check jsonb value type and get to_type value
+    WrapperType create_jsonb_wrapper(const DataTypeJsonb& from_type,
+                                     const DataTypePtr& to_type) const {
+        // Conversion from String through parsing.
+        if (check_and_get_data_type<DataTypeString>(to_type.get())) {
+            return &ConvertImplGenericToString::execute2;
+        }
+
+        switch (to_type->get_type_id()) {
+        case TypeIndex::UInt8:
+            return &ConvertImplFromJsonb<TypeIndex::UInt8, ColumnUInt8>::execute;
+        case TypeIndex::Int8:
+            return &ConvertImplFromJsonb<TypeIndex::Int8, ColumnInt8>::execute;
+        case TypeIndex::Int16:
+            return &ConvertImplFromJsonb<TypeIndex::Int16, ColumnInt16>::execute;
+        case TypeIndex::Int32:
+            return &ConvertImplFromJsonb<TypeIndex::Int32, ColumnInt32>::execute;
+        case TypeIndex::Int64:
+            return &ConvertImplFromJsonb<TypeIndex::Int64, ColumnInt64>::execute;
+        case TypeIndex::Float64:
+            return &ConvertImplFromJsonb<TypeIndex::Float64, ColumnFloat64>::execute;
+        default:
+            return create_unsupport_wrapper(from_type.get_name(), to_type->get_name());
+        }
+
+        return nullptr;
+    }
+
+    // create cresponding jsonb value with type to_type
+    // use jsonb writer to create jsonb value
+    WrapperType create_jsonb_wrapper(const DataTypePtr& from_type,
+                                     const DataTypeJsonb& to_type) const {
+        switch (from_type->get_type_id()) {
+        case TypeIndex::UInt8:
+            return &ConvertImplNumberToJsonb<ColumnUInt8>::execute;
+        case TypeIndex::Int8:
+            return &ConvertImplNumberToJsonb<ColumnInt8>::execute;
+        case TypeIndex::Int16:
+            return &ConvertImplNumberToJsonb<ColumnInt16>::execute;
+        case TypeIndex::Int32:
+            return &ConvertImplNumberToJsonb<ColumnInt32>::execute;
+        case TypeIndex::Int64:
+            return &ConvertImplNumberToJsonb<ColumnInt64>::execute;
+        case TypeIndex::Float64:
+            return &ConvertImplNumberToJsonb<ColumnFloat64>::execute;
+        case TypeIndex::String:
+            return &ConvertImplGenericFromString<ColumnString>::execute;
+        default:
+            return &ConvertImplGenericToJsonb::execute;
+        }
+    }
+
     WrapperType prepare_unpack_dictionaries(const DataTypePtr& from_type,
                                             const DataTypePtr& to_type) const {
         const auto& from_nested = from_type;
@@ -1251,7 +1507,7 @@ private:
 
         if (from_type->only_null()) {
             if (!to_nested->is_nullable()) {
-                LOG(FATAL) << "Cannot convert NULL to a non-nullable type";
+                return create_unsupport_wrapper("Cannot convert NULL to a non-nullable type");
             }
 
             return [](FunctionContext* context, Block& block, const ColumnNumbers&,
@@ -1358,6 +1614,13 @@ private:
         else if (WhichDataType(from_type).is_nothing())
             return create_nothing_wrapper(to_type.get());
 
+        if (from_type->get_type_id() == TypeIndex::JSONB) {
+            return create_jsonb_wrapper(static_cast<const DataTypeJsonb&>(*from_type), to_type);
+        }
+        if (to_type->get_type_id() == TypeIndex::JSONB) {
+            return create_jsonb_wrapper(from_type, static_cast<const DataTypeJsonb&>(*to_type));
+        }
+
         WrapperType ret;
 
         auto make_default_wrapper = [&](const auto& types) -> bool {
@@ -1407,9 +1670,7 @@ private:
             break;
         }
 
-        LOG(FATAL) << fmt::format("Conversion from {} to {} is not supported",
-                                  from_type->get_name(), to_type->get_name());
-        return WrapperType {};
+        return create_unsupport_wrapper(from_type->get_name(), to_type->get_name());
     }
 };
 
@@ -1455,7 +1716,8 @@ protected:
         need_to_be_nullable |= arguments[0].type->is_nullable();
         // 2. from_type is string, to_type is not string
         need_to_be_nullable |= (arguments[0].type->get_type_id() == TypeIndex::String) &&
-                               (type->get_type_id() != TypeIndex::String);
+                               (type->get_type_id() != TypeIndex::String) &&
+                               (type->get_type_id() != TypeIndex::JSONB);
         // 3. from_type is not DateTime/Date, to_type is DateTime/Date
         need_to_be_nullable |= (arguments[0].type->get_type_id() != TypeIndex::Date &&
                                 arguments[0].type->get_type_id() != TypeIndex::DateTime) &&
diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp
index 3d423b053e..9483f13c00 100644
--- a/be/src/vec/sink/vmysql_result_writer.cpp
+++ b/be/src/vec/sink/vmysql_result_writer.cpp
@@ -119,10 +119,8 @@ Status VMysqlResultWriter::_add_one_column(const ColumnPtr& column_ptr,
                         buf_ret = _buffer.push_null();
                     }
                 } else {
-                    JsonbToJson toStr;
-                    std::string json_str = toStr.jsonb_to_string(
-                            JsonbDocument::createDocument(json_val.data, json_val.size)
-                                    ->getValue());
+                    std::string json_str =
+                            JsonbToJson::jsonb_to_json_string(json_val.data, json_val.size);
                     buf_ret = _buffer.push_string(json_str.c_str(), json_str.size());
                 }
             }
diff --git a/be/test/vec/core/column_jsonb_test.cpp b/be/test/vec/core/column_jsonb_test.cpp
index 663509e257..ca9aeac809 100644
--- a/be/test/vec/core/column_jsonb_test.cpp
+++ b/be/test/vec/core/column_jsonb_test.cpp
@@ -50,9 +50,7 @@ TEST(ColumnJsonbTest, SingleValueTest) {
 
     for (size_t i = 0; i < offs.size(); i++) {
         auto v = data_column->get_data_at(offs[i]);
-        JsonbToJson toStr;
-        std::string json_str =
-                toStr.jsonb_to_string(JsonbDocument::createDocument(v.data, v.size)->getValue());
+        std::string json_str = JsonbToJson::jsonb_to_json_string(v.data, v.size);
         EXPECT_EQ(vals[i], json_str);
     }
 }
diff --git a/be/test/vec/function/function_jsonb_test.cpp b/be/test/vec/function/function_jsonb_test.cpp
index a31bcc8939..4898cea917 100644
--- a/be/test/vec/function/function_jsonb_test.cpp
+++ b/be/test/vec/function/function_jsonb_test.cpp
@@ -27,9 +27,10 @@ using namespace ut_type;
 
 TEST(FunctionJsonbTEST, JsonbParseTest) {
     std::string func_name = "jsonb_parse";
-    InputTypeSet input_types = {TypeIndex::String};
+    InputTypeSet input_types = {Nullable {TypeIndex::String}};
 
     DataSet data_set_valid = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -50,51 +51,45 @@ TEST(FunctionJsonbTEST, JsonbParseTest) {
              STRING(R"([{"k1":"v41","k2":400},1,"a",3.14])")}, // complex array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_valid);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_valid);
 
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseErrorToNullTest) {
     std::string func_name = "jsonb_parse_error_to_null";
-    InputTypeSet input_types = {TypeIndex::String};
+    InputTypeSet input_types = {Nullable {TypeIndex::String}};
 
     DataSet data_set = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -121,15 +116,15 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToNullTest) {
             {{STRING("[123, abc]")}, Null()}                   // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseErrorToValueTest) {
     std::string func_name = "jsonb_parse_error_to_value";
-    InputTypeSet input_types = {TypeIndex::String, TypeIndex::String};
+    InputTypeSet input_types = {Nullable {TypeIndex::String}, TypeIndex::String};
 
     DataSet data_set = {
+            {{Null(), STRING("{}")}, Null()},
             {{STRING("null"), STRING("{}")}, STRING("null")},
             {{STRING("true"), STRING("{}")}, STRING("true")},
             {{STRING("false"), STRING("{}")}, STRING("false")},
@@ -159,15 +154,15 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToValueTest) {
              STRING(R"([123,"abc"])")} // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseErrorToInvalidTest) {
     std::string func_name = "jsonb_parse_error_to_invalid";
-    InputTypeSet input_types = {TypeIndex::String};
+    InputTypeSet input_types = {Nullable {TypeIndex::String}};
 
     DataSet data_set = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -194,8 +189,7 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToInvalidTest) {
             {{STRING("[123, abc]")}, STRING("")}               // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNullableTest) {
@@ -203,6 +197,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableTest) {
     InputTypeSet input_types = {TypeIndex::String};
 
     DataSet data_set_valid = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -223,44 +218,37 @@ TEST(FunctionJsonbTEST, JsonbParseNullableTest) {
              STRING(R"([{"k1":"v41","k2":400},1,"a",3.14])")}, // complex array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_valid);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_valid);
 
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNullableErrorToNullTest) {
@@ -268,6 +256,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToNullTest) {
     InputTypeSet input_types = {TypeIndex::String};
 
     DataSet data_set = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -294,8 +283,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToNullTest) {
             {{STRING("[123, abc]")}, Null()}                   // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNullableErrorToValueTest) {
@@ -303,6 +291,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToValueTest) {
     InputTypeSet input_types = {TypeIndex::String, TypeIndex::String};
 
     DataSet data_set = {
+            {{Null(), STRING("{}")}, Null()},
             {{STRING("null"), STRING("{}")}, STRING("null")},
             {{STRING("true"), STRING("{}")}, STRING("true")},
             {{STRING("false"), STRING("{}")}, STRING("false")},
@@ -332,8 +321,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToValueTest) {
              STRING(R"([123,"abc"])")} // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNullableErrorToInvalidTest) {
@@ -341,6 +329,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToInvalidTest) {
     InputTypeSet input_types = {TypeIndex::String};
 
     DataSet data_set = {
+            {{Null()}, Null()},
             {{STRING("null")}, STRING("null")},
             {{STRING("true")}, STRING("true")},
             {{STRING("false")}, STRING("false")},
@@ -367,8 +356,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToInvalidTest) {
             {{STRING("[123, abc]")}, STRING("")}               // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNotnullTest) {
@@ -396,44 +384,37 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullTest) {
              STRING(R"([{"k1":"v41","k2":400},1,"a",3.14])")}, // complex array
     };
 
-    auto st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_valid);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_valid);
 
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
-    EXPECT_NE(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToValueTest) {
@@ -470,8 +451,7 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToValueTest) {
              STRING(R"([123,"abc"])")} // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToInvalidTest) {
@@ -505,8 +485,7 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToInvalidTest) {
             {{STRING("[123, abc]")}, STRING("")}               // invalid array
     };
 
-    auto st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, false>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbExtractTest) {
@@ -515,6 +494,7 @@ TEST(FunctionJsonbTEST, JsonbExtractTest) {
 
     // jsonb_extract root
     DataSet data_set = {
+            {{Null(), STRING("$")}, Null()},
             {{STRING("null"), STRING("$")}, STRING("null")},
             {{STRING("true"), STRING("$")}, STRING("true")},
             {{STRING("false"), STRING("$")}, STRING("false")},
@@ -537,11 +517,11 @@ TEST(FunctionJsonbTEST, JsonbExtractTest) {
              STRING(R"([{"k1":"v41","k2":400},1,"a",3.14])")}, // complex array
     };
 
-    auto st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 
     // jsonb_extract obejct
     data_set = {
+            {{Null(), STRING("$.k1")}, Null()},
             {{STRING("null"), STRING("$.k1")}, Null()},
             {{STRING("true"), STRING("$.k1")}, Null()},
             {{STRING("false"), STRING("$.k1")}, Null()},
@@ -562,11 +542,11 @@ TEST(FunctionJsonbTEST, JsonbExtractTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 
     // jsonb_extract array
     data_set = {
+            {{Null(), STRING("$[0]")}, Null()},
             {{STRING("null"), STRING("$[0]")}, Null()},
             {{STRING("true"), STRING("$[0]")}, Null()},
             {{STRING("false"), STRING("$[0]")}, Null()},
@@ -623,11 +603,11 @@ TEST(FunctionJsonbTEST, JsonbExtractTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 
     // jsonb_extract $[0].k1
     data_set = {
+            {{Null(), STRING("$[0].k1")}, Null()},
             {{STRING("null"), STRING("$[0].k1")}, Null()},
             {{STRING("true"), STRING("$[0].k1")}, Null()},
             {{STRING("false"), STRING("$[0].k1")}, Null()},
@@ -648,8 +628,7 @@ TEST(FunctionJsonbTEST, JsonbExtractTest) {
              STRING(R"("v41")")}, // complex array
     };
 
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
@@ -658,6 +637,7 @@ TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
 
     // jsonb_extract root
     DataSet data_set = {
+            {{Null(), STRING("$")}, Null()},
             {{STRING("null"), STRING("$")}, Null()},
             {{STRING("true"), STRING("$")}, Null()},
             {{STRING("false"), STRING("$")}, Null()},
@@ -678,11 +658,11 @@ TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
              Null()}, // complex array
     };
 
-    auto st = check_function<DataTypeString, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeString, true>(func_name, input_types, data_set);
 
     // jsonb_extract obejct
     data_set = {
+            {{Null(), STRING("$.k1")}, Null()},
             {{STRING("null"), STRING("$.k1")}, Null()},
             {{STRING("true"), STRING("$.k1")}, Null()},
             {{STRING("false"), STRING("$.k1")}, Null()},
@@ -703,11 +683,11 @@ TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeString, true>(func_name, input_types, data_set);
 
     // jsonb_extract array
     data_set = {
+            {{Null(), STRING("$[0]")}, Null()},
             {{STRING("null"), STRING("$[0]")}, Null()},
             {{STRING("true"), STRING("$[0]")}, Null()},
             {{STRING("false"), STRING("$[0]")}, Null()},
@@ -764,11 +744,11 @@ TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeString, true>(func_name, input_types, data_set);
 
     // jsonb_extract $[0].k1
     data_set = {
+            {{Null(), STRING("$[0].k1")}, Null()},
             {{STRING("null"), STRING("$[0].k1")}, Null()},
             {{STRING("true"), STRING("$[0].k1")}, Null()},
             {{STRING("false"), STRING("$[0].k1")}, Null()},
@@ -789,8 +769,7 @@ TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
              STRING("v41")}, // complex array
     };
 
-    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeString, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
@@ -799,6 +778,7 @@ TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
 
     // jsonb_extract root
     DataSet data_set = {
+            {{Null(), STRING("$")}, Null()},
             {{STRING("null"), STRING("$")}, Null()},
             {{STRING("true"), STRING("$")}, Null()},
             {{STRING("false"), STRING("$")}, Null()},
@@ -819,11 +799,11 @@ TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
              Null()}, // complex array
     };
 
-    auto st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt32, true>(func_name, input_types, data_set);
 
     // jsonb_extract obejct
     data_set = {
+            {{Null(), STRING("$.k1")}, Null()},
             {{STRING("null"), STRING("$.k1")}, Null()},
             {{STRING("true"), STRING("$.k1")}, Null()},
             {{STRING("false"), STRING("$.k1")}, Null()},
@@ -844,11 +824,11 @@ TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt32, true>(func_name, input_types, data_set);
 
     // jsonb_extract array
     data_set = {
+            {{Null(), STRING("$[0]")}, Null()},
             {{STRING("null"), STRING("$[0]")}, Null()},
             {{STRING("true"), STRING("$[0]")}, Null()},
             {{STRING("false"), STRING("$[0]")}, Null()},
@@ -905,11 +885,11 @@ TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt32, true>(func_name, input_types, data_set);
 
     // jsonb_extract $[0].k1
     data_set = {
+            {{Null(), STRING("$[0].k1")}, Null()},
             {{STRING("null"), STRING("$[0].k1")}, Null()},
             {{STRING("true"), STRING("$[0].k1")}, Null()},
             {{STRING("false"), STRING("$[0].k1")}, Null()},
@@ -932,8 +912,7 @@ TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
              INT(400)}, // complex array
     };
 
-    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt32, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
@@ -942,6 +921,7 @@ TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
 
     // jsonb_extract root
     DataSet data_set = {
+            {{Null(), STRING("$")}, Null()},
             {{STRING("null"), STRING("$")}, Null()},
             {{STRING("true"), STRING("$")}, Null()},
             {{STRING("false"), STRING("$")}, Null()},
@@ -962,11 +942,11 @@ TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
              Null()}, // complex array
     };
 
-    auto st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt64, true>(func_name, input_types, data_set);
 
     // jsonb_extract obejct
     data_set = {
+            {{Null(), STRING("$.k1")}, Null()},
             {{STRING("null"), STRING("$.k1")}, Null()},
             {{STRING("true"), STRING("$.k1")}, Null()},
             {{STRING("false"), STRING("$.k1")}, Null()},
@@ -987,11 +967,11 @@ TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt64, true>(func_name, input_types, data_set);
 
     // jsonb_extract array
     data_set = {
+            {{Null(), STRING("$[0]")}, Null()},
             {{STRING("null"), STRING("$[0]")}, Null()},
             {{STRING("true"), STRING("$[0]")}, Null()},
             {{STRING("false"), STRING("$[0]")}, Null()},
@@ -1048,11 +1028,11 @@ TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt64, true>(func_name, input_types, data_set);
 
     // jsonb_extract $[0].k1
     data_set = {
+            {{Null(), STRING("$[0].k1")}, Null()},
             {{STRING("null"), STRING("$[0].k1")}, Null()},
             {{STRING("true"), STRING("$[0].k1")}, Null()},
             {{STRING("false"), STRING("$[0].k1")}, Null()},
@@ -1075,8 +1055,7 @@ TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
              BIGINT(400)}, // complex array
     };
 
-    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeInt64, true>(func_name, input_types, data_set);
 }
 
 TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
@@ -1085,6 +1064,7 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
 
     // jsonb_extract root
     DataSet data_set = {
+            {{Null(), STRING("$")}, Null()},
             {{STRING("null"), STRING("$")}, Null()},
             {{STRING("true"), STRING("$")}, Null()},
             {{STRING("false"), STRING("$")}, Null()},
@@ -1105,8 +1085,7 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
              Null()}, // complex array
     };
 
-    auto st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
 
     // jsonb_extract obejct
     data_set = {
@@ -1130,8 +1109,7 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
 
     // jsonb_extract array
     data_set = {
@@ -1191,8 +1169,7 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
              Null()}, // complex array
     };
 
-    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
 
     // jsonb_extract $[0].k1
     data_set = {
@@ -1218,8 +1195,230 @@ TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
              DOUBLE(400)}, // complex array
     };
 
-    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
-    EXPECT_EQ(Status::OK(), st);
+    check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+}
+
+TEST(FunctionJsonbTEST, JsonbCastToOtherTest) {
+    std::string func_name = "CAST";
+    InputTypeSet input_types = {Nullable {TypeIndex::JSONB}, ConstedNotnull {TypeIndex::String}};
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("UInt8")},
+             Null()}, // complex array
+    };
+
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeUInt8, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int8")},
+             Null()}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeInt8, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int16")},
+             Null()}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeInt16, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int32")},
+             Null()}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeInt32, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Int64")},
+             Null()}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeInt64, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("Float64")},
+             Null()}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeFloat64, true>(func_name, input_types, const_dataset);
+    }
+
+    // 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(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"])")}, // 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])")}, // complex array
+    };
+    for (const auto& row : data_set) {
+        DataSet const_dataset = {row};
+        check_function<DataTypeString, true>(func_name, input_types, const_dataset);
+    }
+}
+
+TEST(FunctionJsonbTEST, JsonbCastFromOtherTest) {
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
+            {{{BOOLEAN(1), STRING("Json")}, STRING("true")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::UInt8}, ConstedNotnull {TypeIndex::String}},
+            {{{BOOLEAN(0), STRING("Json")}, STRING("false")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::Int8}, ConstedNotnull {TypeIndex::String}},
+            {{{TINYINT(100), STRING("Json")}, STRING("100")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::Int16}, ConstedNotnull {TypeIndex::String}},
+            {{{SMALLINT(10000), STRING("Json")}, STRING("10000")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::Int32}, ConstedNotnull {TypeIndex::String}},
+            {{{INT(1000000000), STRING("Json")}, STRING("1000000000")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::Int64}, ConstedNotnull {TypeIndex::String}},
+            {{{BIGINT(1152921504606846976), STRING("Json")}, STRING("1152921504606846976")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::Float64}, ConstedNotnull {TypeIndex::String}},
+            {{{DOUBLE(6.18), STRING("Json")}, STRING("6.18")}});
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
+            {{{STRING(R"(abcd)"), STRING("Json")}, Null()}}); // should fail
+    check_function<DataTypeJsonb, true>(
+            "CAST", {Nullable {TypeIndex::String}, ConstedNotnull {TypeIndex::String}},
+            {{{STRING(R"("abcd")"), STRING("Json")}, STRING(R"("abcd")")}});
 }
 
 } // namespace doris::vectorized
diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp
index 4c223299bc..a2e002acec 100644
--- a/be/test/vec/function/function_test_util.cpp
+++ b/be/test/vec/function/function_test_util.cpp
@@ -58,6 +58,15 @@ size_t type_index_to_data_type(const std::vector<std::any>& input_types, size_t
     TypeIndex tp;
     if (input_types[index].type() == typeid(Consted)) {
         tp = std::any_cast<Consted>(input_types[index]).tp;
+    } else if (input_types[index].type() == typeid(ConstedNotnull)) {
+        tp = std::any_cast<ConstedNotnull>(input_types[index]).tp;
+        ut_desc.is_nullable = false;
+    } else if (input_types[index].type() == typeid(Nullable)) {
+        tp = std::any_cast<Nullable>(input_types[index]).tp;
+        ut_desc.is_nullable = true;
+    } else if (input_types[index].type() == typeid(Notnull)) {
+        tp = std::any_cast<Notnull>(input_types[index]).tp;
+        ut_desc.is_nullable = false;
     } else {
         tp = std::any_cast<TypeIndex>(input_types[index]);
     }
@@ -75,6 +84,10 @@ size_t type_index_to_data_type(const std::vector<std::any>& input_types, size_t
         desc.type = doris_udf::FunctionContext::TYPE_OBJECT;
         type = std::make_shared<DataTypeBitMap>();
         return 1;
+    case TypeIndex::UInt8:
+        desc.type = doris_udf::FunctionContext::TYPE_BOOLEAN;
+        type = std::make_shared<DataTypeUInt8>();
+        return 1;
     case TypeIndex::Int8:
         desc.type = doris_udf::FunctionContext::TYPE_TINYINT;
         type = std::make_shared<DataTypeInt8>();
@@ -156,7 +169,8 @@ bool parse_ut_data_type(const std::vector<std::any>& input_types, ut_type::UTDat
     descs.reserve(input_types.size());
     for (size_t i = 0; i < input_types.size();) {
         ut_type::UTDataTypeDesc desc;
-        if (input_types[i].type() == typeid(Consted)) {
+        if (input_types[i].type() == typeid(Consted) ||
+            input_types[i].type() == typeid(ConstedNotnull)) {
             desc.is_const = true;
         }
         size_t res = type_index_to_data_type(input_types, i, desc, desc.data_type);
@@ -214,6 +228,9 @@ bool insert_cell(MutableColumnPtr& column, DataTypePtr type_ptr, const std::any&
     } else if (type.idx == TypeIndex::BitMap) {
         BitmapValue* bitmap = std::any_cast<BitmapValue*>(cell);
         column->insert_data((char*)bitmap, sizeof(BitmapValue));
+    } else if (type.is_uint8()) {
+        auto value = std::any_cast<ut_type::BOOLEAN>(cell);
+        column->insert_data(reinterpret_cast<char*>(&value), 0);
     } else if (type.is_int8()) {
         auto value = std::any_cast<ut_type::TINYINT>(cell);
         column->insert_data(reinterpret_cast<char*>(&value), 0);
diff --git a/be/test/vec/function/function_test_util.h b/be/test/vec/function/function_test_util.h
index 5d652d9b6c..e48c4695fa 100644
--- a/be/test/vec/function/function_test_util.h
+++ b/be/test/vec/function/function_test_util.h
@@ -49,7 +49,20 @@ int64_t str_to_date_time(std::string datetime_str, bool data_time = true);
 uint32_t str_to_date_v2(std::string datetime_str, std::string datetime_format);
 uint64_t str_to_datetime_v2(std::string datetime_str, std::string datetime_format);
 
+struct Nullable {
+    TypeIndex tp;
+};
+
+struct Notnull {
+    TypeIndex tp;
+};
+
+struct ConstedNotnull {
+    TypeIndex tp;
+};
+
 namespace ut_type {
+using BOOLEAN = uint8_t;
 using TINYINT = int8_t;
 using SMALLINT = int16_t;
 using INT = int32_t;
@@ -240,10 +253,12 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty
     block.insert({nullptr, return_type, "result"});
 
     auto result = block.columns() - 1;
+    auto st = func->execute(fn_ctx, block, arguments, result, row_size);
     if (expect_fail) {
-        RETURN_IF_ERROR(func->execute(fn_ctx, block, arguments, result, row_size));
+        EXPECT_NE(Status::OK(), st);
+        return st;
     } else {
-        func->execute(fn_ctx, block, arguments, result, row_size);
+        EXPECT_EQ(Status::OK(), st);
     }
 
     func->close(fn_ctx, FunctionContext::THREAD_LOCAL);
@@ -263,10 +278,8 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty
                     EXPECT_EQ(0, s.size) << " invalid result size should be 0 at row " << i;
                 } else {
                     // convert jsonb binary value to json string to compare with expected json text
-                    JsonbToJson to_json;
-                    doris::JsonbValue* val =
-                            doris::JsonbDocument::createDocument(s.data, s.size)->getValue();
-                    EXPECT_EQ(to_json.jsonb_to_string(val), expect_data) << " at row " << i;
+                    EXPECT_EQ(expect_data, JsonbToJson::jsonb_to_json_string(s.data, s.size))
+                            << " at row " << i;
                 }
             } else {
                 Field field;


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