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/19 00:44:15 UTC

[doris] branch master updated: [feature](jsonb) jsonb functions (#13366)

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 755a946516 [feature](jsonb) jsonb functions (#13366)
755a946516 is described below

commit 755a9465169dfdc8074e2e7ed7d456bfed5766d8
Author: Kang <kx...@gmail.com>
AuthorDate: Wed Oct 19 08:44:08 2022 +0800

    [feature](jsonb) jsonb functions (#13366)
    
    Issue Number: Step3 of DSIP-016: Support JSON type
---
 be/src/util/jsonb_document.h                 | 122 ++++-
 be/src/util/jsonb_writer.h                   |   3 +-
 be/src/vec/functions/function_jsonb.cpp      | 363 ++++++++++++-
 be/test/vec/function/function_jsonb_test.cpp | 771 ++++++++++++++++++++++++++-
 be/test/vec/function/function_test_util.cpp  |   9 +
 be/test/vec/function/function_test_util.h    |  28 +-
 gensrc/script/doris_builtins_functions.py    |  56 ++
 7 files changed, 1284 insertions(+), 68 deletions(-)

diff --git a/be/src/util/jsonb_document.h b/be/src/util/jsonb_document.h
index 94b3c9cc8a..98f98b8377 100644
--- a/be/src/util/jsonb_document.h
+++ b/be/src/util/jsonb_document.h
@@ -72,6 +72,8 @@
 
 #include <limits>
 
+// #include "util/string_parser.hpp"
+
 namespace doris {
 
 #pragma pack(push, 1)
@@ -348,6 +350,34 @@ public:
 
     JsonbType type() const { return type_; }
 
+    const char* typeName() const {
+        switch (type_) {
+        case JsonbType::T_Null:
+            return "null";
+        case JsonbType::T_True:
+        case JsonbType::T_False:
+            return "bool";
+        case JsonbType::T_Int8:
+        case JsonbType::T_Int16:
+        case JsonbType::T_Int32:
+            return "int";
+        case JsonbType::T_Int64:
+            return "bigint";
+        case JsonbType::T_Double:
+            return "double";
+        case JsonbType::T_String:
+            return "string";
+        case JsonbType::T_Binary:
+            return "binary";
+        case JsonbType::T_Object:
+            return "object";
+        case JsonbType::T_Array:
+            return "array";
+        default:
+            return "unknown";
+        }
+    }
+
     // size of the total packed bytes
     unsigned int numPackedBytes() const;
 
@@ -997,7 +1027,20 @@ inline const char* JsonbValue::getValuePtr() const {
 
 inline JsonbValue* JsonbValue::findPath(const char* key_path, unsigned int kp_len,
                                         const char* delim = ".", hDictFind handler = nullptr) {
-    if (!key_path || !kp_len) return nullptr;
+    if (!key_path) return nullptr;
+    if (kp_len == 0) return this;
+
+    // skip $ and . at beginning
+    if (kp_len > 0 && *key_path == '$') {
+        key_path++;
+        kp_len--;
+        if (kp_len > 0 && *key_path == '.') {
+            key_path++;
+            kp_len--;
+        }
+    }
+
+    if (kp_len == 0) return this;
 
     if (!delim) delim = "."; // default delimiter
 
@@ -1008,37 +1051,66 @@ inline JsonbValue* JsonbValue::findPath(const char* key_path, unsigned int kp_le
     while (pval && key_path < fence) {
         const char* key = key_path;
         unsigned int klen = 0;
-        // find the current key
-        for (; key_path != fence && *key_path != *delim; ++key_path, ++klen)
-            ;
-
-        if (!klen) return nullptr;
-
-        switch (pval->type_) {
-        case JsonbType::T_Object: {
-            pval = ((ObjectVal*)pval)->find(key, klen, handler);
-            break;
+        const char* left_bracket = nullptr;
+        const char* right_bracket = nullptr;
+        size_t idx_len = 0;
+        // find the current key and [] bracket position
+        for (; key_path != fence && *key_path != *delim; ++key_path, ++klen) {
+            if ('[' == *key_path) {
+                left_bracket = key_path;
+            } else if (']' == *key_path) {
+                right_bracket = key_path;
+            }
         }
 
-        case JsonbType::T_Array: {
-            // parse string into an integer (array index)
-            if (klen >= sizeof(idx_buf)) return nullptr;
+        // check brackets and array index length
+        if (left_bracket || right_bracket) {
+            if (!left_bracket || !right_bracket) {
+                return nullptr;
+            }
+            // check the last char is ]
+            if (key + klen - 1 != right_bracket) {
+                return nullptr;
+            }
+            // the part before left_bracket is object key
+            klen = left_bracket - key;
+            // the part between left_bracket and right_bracket is array index
+            idx_len = right_bracket - left_bracket - 1;
+        }
 
-            memcpy(idx_buf, key, klen);
-            idx_buf[klen] = 0;
+        if (!klen && !idx_len) return nullptr;
 
-            char* end = nullptr;
-            int index = (int)strtol(idx_buf, &end, 10);
-            if (end && !*end)
-                pval = ((ArrayVal*)pval)->get(index);
-            else
-                // incorrect index string
+        // get value of key in object
+        if (klen) {
+            if (LIKELY(pval->type_ == JsonbType::T_Object)) {
+                pval = ((ObjectVal*)pval)->find(key, klen, handler);
+                if (!pval) return nullptr;
+            } else {
                 return nullptr;
-            break;
+            }
         }
 
-        default:
-            return nullptr;
+        // get value at idx in array
+        if (idx_len) {
+            if (LIKELY(pval->type_ == JsonbType::T_Array)) {
+                if (idx_len >= sizeof(idx_buf)) return nullptr;
+                memcpy(idx_buf, left_bracket + 1, idx_len);
+                idx_buf[idx_len] = 0;
+
+                char* end = nullptr;
+                int index = (int)strtol(idx_buf, &end, 10);
+                if (end && !*end)
+                    pval = ((ArrayVal*)pval)->get(index);
+                else
+                    // incorrect index string
+                    return nullptr;
+
+                // doris::StringParser::ParseResult parse_result;
+                // int index = doris::StringParser::string_to_int<int>(left_bracket + 1, idx_len, &parse_result);
+                // if (parse_result == doris::StringParser::ParseResult::PARSE_SUCCESS)
+            } else {
+                return nullptr;
+            }
         }
 
         // skip the delimiter
diff --git a/be/src/util/jsonb_writer.h b/be/src/util/jsonb_writer.h
index ccce7d98e0..fa3098cb55 100644
--- a/be/src/util/jsonb_writer.h
+++ b/be/src/util/jsonb_writer.h
@@ -106,7 +106,8 @@ public:
     }
 
     uint32_t writeValue(const JsonbValue* value) {
-        if (!stack_.empty() && verifyValueState()) {
+        if ((first_ && stack_.empty()) || (!stack_.empty() && verifyValueState())) {
+            if (!writeFirstHeader()) return 0;
             os_->write((char*)value, value->numPackedBytes());
             kvState_ = WS_Value;
             return value->size();
diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp
index 39cf25de81..a480735ca4 100644
--- a/be/src/vec/functions/function_jsonb.cpp
+++ b/be/src/vec/functions/function_jsonb.cpp
@@ -18,7 +18,6 @@
 #include <boost/token_functions.hpp>
 #include <vector>
 
-#include "exprs/json_functions.h"
 #include "util/string_parser.hpp"
 #include "util/string_util.h"
 #include "vec/columns/column.h"
@@ -290,6 +289,357 @@ using FunctionJsonbParseNotnullErrorValue =
 using FunctionJsonbParseNotnullErrorInvalid =
         FunctionJsonbParseBase<NullalbeMode::NOT_NULL, JsonbParseErrorMode::RETURN_INVALID>;
 
+// func(json,string) -> nullable(type)
+template <typename Impl>
+class FunctionJsonbExtract : public IFunction {
+public:
+    static constexpr auto name = Impl::name;
+    static FunctionPtr create() { return std::make_shared<FunctionJsonbExtract>(); }
+    String get_name() const override { return name; }
+    size_t get_number_of_arguments() const override { return 2; }
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+        return make_nullable(std::make_shared<typename Impl::ReturnType>());
+    }
+
+    bool use_default_implementation_for_constants() const override { return true; }
+
+    Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) override {
+        auto null_map = ColumnUInt8::create(input_rows_count, 0);
+        DCHECK_EQ(arguments.size(), 2);
+        ColumnPtr argument_columns[2];
+        for (int i = 0; i < 2; ++i) {
+            argument_columns[i] =
+                    block.get_by_position(arguments[i]).column->convert_to_full_column_if_const();
+            if (auto* nullable = check_and_get_column<ColumnNullable>(*argument_columns[i])) {
+                // Danger: Here must dispose the null map data first! Because
+                // argument_columns[i]=nullable->get_nested_column_ptr(); will release the mem
+                // of column nullable mem of null map
+                VectorizedUtils::update_null_map(null_map->get_data(),
+                                                 nullable->get_null_map_data());
+                argument_columns[i] = nullable->get_nested_column_ptr();
+            }
+        }
+
+        auto res = Impl::ColumnType::create();
+
+        auto jsonb_data_column = assert_cast<const ColumnJsonb*>(argument_columns[0].get());
+        auto jsonb_path_column = assert_cast<const ColumnString*>(argument_columns[1].get());
+
+        auto& ldata = jsonb_data_column->get_chars();
+        auto& loffsets = jsonb_data_column->get_offsets();
+
+        auto& rdata = jsonb_path_column->get_chars();
+        auto& roffsets = jsonb_path_column->get_offsets();
+
+        // execute Impl
+        if constexpr (std::is_same_v<typename Impl::ReturnType, DataTypeString> ||
+                      std::is_same_v<typename Impl::ReturnType, DataTypeJsonb>) {
+            auto& res_data = res->get_chars();
+            auto& res_offsets = res->get_offsets();
+            Impl::vector_vector(context, ldata, loffsets, rdata, roffsets, res_data, res_offsets,
+                                null_map->get_data());
+        } else {
+            Impl::vector_vector(context, ldata, loffsets, rdata, roffsets, res->get_data(),
+                                null_map->get_data());
+        }
+        block.get_by_position(result).column =
+                ColumnNullable::create(std::move(res), std::move(null_map));
+        return Status::OK();
+    }
+};
+
+template <typename ValueType>
+struct JsonbExtractStringImpl {
+    using ReturnType = typename ValueType::ReturnType;
+    using ColumnType = typename ValueType::ColumnType;
+    static const bool only_check_exists = ValueType::only_check_exists;
+
+    // for jsonb_extract_string
+    static void vector_vector(FunctionContext* context, const ColumnString::Chars& ldata,
+                              const ColumnString::Offsets& loffsets,
+                              const ColumnString::Chars& rdata,
+                              const ColumnString::Offsets& roffsets, ColumnString::Chars& res_data,
+                              ColumnString::Offsets& res_offsets, NullMap& null_map) {
+        size_t input_rows_count = loffsets.size();
+        res_offsets.resize(input_rows_count);
+
+        std::unique_ptr<JsonbWriter> writer;
+        if constexpr (std::is_same_v<DataTypeJsonb, ReturnType>) {
+            writer.reset(new JsonbWriter());
+        }
+
+        for (size_t i = 0; i < input_rows_count; ++i) {
+            int l_size = loffsets[i] - loffsets[i - 1] - 1;
+            const auto l_raw = reinterpret_cast<const char*>(&ldata[loffsets[i - 1]]);
+
+            int r_size = roffsets[i] - roffsets[i - 1];
+            const auto r_raw = reinterpret_cast<const char*>(&rdata[roffsets[i - 1]]);
+            String path(r_raw, r_size);
+
+            if (null_map[i]) {
+                StringOP::push_null_string(i, res_data, res_offsets, null_map);
+                continue;
+            }
+
+            // doc is NOT necessary to be deleted since JsonbDocument will not allocate memory
+            JsonbDocument* doc = JsonbDocument::createDocument(l_raw, l_size);
+            if (UNLIKELY(!doc || !doc->getValue())) {
+                StringOP::push_null_string(i, res_data, res_offsets, null_map);
+                continue;
+            }
+
+            // value is NOT necessary to be deleted since JsonbValue will not allocate memory
+            JsonbValue* value = doc->getValue()->findPath(r_raw, r_size, ".", nullptr);
+            if (UNLIKELY(!value)) {
+                StringOP::push_null_string(i, res_data, res_offsets, null_map);
+                continue;
+            }
+
+            if constexpr (ValueType::only_get_type) {
+                StringOP::push_value_string(std::string_view(value->typeName()), i, res_data,
+                                            res_offsets);
+                continue;
+            }
+
+            if constexpr (std::is_same_v<DataTypeJsonb, ReturnType>) {
+                writer->reset();
+                writer->writeValue(value);
+                // StringOP::push_value_string(
+                //     std::string_view(writer->getOutput()->getBuffer(), writer->getOutput()->getSize()),
+                //     i, res_data, res_offsets);
+                res_data.insert(writer->getOutput()->getBuffer(),
+                                writer->getOutput()->getBuffer() + writer->getOutput()->getSize());
+                res_data.push_back('\0');
+                res_offsets[i] = res_data.size();
+            } else {
+                if (LIKELY(value->isString())) {
+                    auto str_value = (JsonbStringVal*)value;
+                    StringOP::push_value_string(
+                            std::string_view(str_value->getBlob(), str_value->length()), i,
+                            res_data, res_offsets);
+                } else {
+                    StringOP::push_null_string(i, res_data, res_offsets, null_map);
+                    continue;
+                }
+            }
+        }
+    }
+};
+
+template <typename ValueType>
+struct JsonbExtractImpl {
+    using ReturnType = typename ValueType::ReturnType;
+    using ColumnType = typename ValueType::ColumnType;
+    using Container = typename ColumnType::Container;
+    static const bool only_check_exists = ValueType::only_check_exists;
+
+    // for jsonb_extract_int/int64/double
+    static void vector_vector(FunctionContext* context, const ColumnString::Chars& ldata,
+                              const ColumnString::Offsets& loffsets,
+                              const ColumnString::Chars& rdata,
+                              const ColumnString::Offsets& roffsets, Container& res,
+                              NullMap& null_map) {
+        size_t size = loffsets.size();
+        res.resize(size);
+
+        for (size_t i = 0; i < loffsets.size(); i++) {
+            if constexpr (only_check_exists) {
+                res[i] = 0;
+            }
+
+            const char* l_raw_str = reinterpret_cast<const char*>(&ldata[loffsets[i - 1]]);
+            int l_str_size = loffsets[i] - loffsets[i - 1] - 1;
+
+            const char* r_raw_str = reinterpret_cast<const char*>(&rdata[roffsets[i - 1]]);
+            int r_str_size = roffsets[i] - roffsets[i - 1];
+
+            if (null_map[i]) {
+                res[i] = 0;
+                continue;
+            }
+
+            // doc is NOT necessary to be deleted since JsonbDocument will not allocate memory
+            JsonbDocument* doc = JsonbDocument::createDocument(l_raw_str, l_str_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()->findPath(r_raw_str, r_str_size, ".", nullptr);
+            if (UNLIKELY(!value)) {
+                null_map[i] = 1;
+                res[i] = 0;
+                continue;
+            }
+
+            // if only check path exists, it's true here and skip check value
+            if constexpr (only_check_exists) {
+                res[i] = 1;
+                continue;
+            }
+
+            if constexpr (std::is_same_v<void, typename ValueType::T>) {
+                if (value->isNull()) {
+                    res[i] = 1;
+                } else {
+                    res[i] = 0;
+                }
+            } else if constexpr (std::is_same_v<bool, typename ValueType::T>) {
+                if (value->isTrue()) {
+                    res[i] = 1;
+                } else if (value->isFalse()) {
+                    res[i] = 0;
+                } else {
+                    null_map[i] = 1;
+                    res[i] = 0;
+                }
+            } else if constexpr (std::is_same_v<int32_t, typename ValueType::T>) {
+                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 (std::is_same_v<int64_t, typename ValueType::T>) {
+                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 (std::is_same_v<double, typename ValueType::T>) {
+                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) << "unexpected type ";
+            }
+        }
+    }
+};
+
+struct JsonbTypeExists {
+    using T = uint8_t;
+    using ReturnType = DataTypeUInt8;
+    using ColumnType = ColumnVector<T>;
+    static const bool only_check_exists = true;
+};
+
+struct JsonbTypeNull {
+    using T = void;
+    using ReturnType = DataTypeUInt8;
+    using ColumnType = ColumnVector<uint8_t>;
+    static const bool only_check_exists = false;
+};
+
+struct JsonbTypeBool {
+    using T = bool;
+    using ReturnType = DataTypeUInt8;
+    using ColumnType = ColumnVector<uint8_t>;
+    static const bool only_check_exists = false;
+};
+
+struct JsonbTypeInt {
+    using T = int32_t;
+    using ReturnType = DataTypeInt32;
+    using ColumnType = ColumnVector<T>;
+    static const bool only_check_exists = false;
+};
+
+struct JsonbTypeInt64 {
+    using T = int64_t;
+    using ReturnType = DataTypeInt64;
+    using ColumnType = ColumnVector<T>;
+    static const bool only_check_exists = false;
+};
+
+struct JsonbTypeDouble {
+    using T = double;
+    using ReturnType = DataTypeFloat64;
+    using ColumnType = ColumnVector<T>;
+    static const bool only_check_exists = false;
+};
+
+struct JsonbTypeString {
+    using T = std::string;
+    using ReturnType = DataTypeString;
+    using ColumnType = ColumnString;
+    static const bool only_check_exists = false;
+    static const bool only_get_type = false;
+};
+
+struct JsonbTypeJson {
+    using T = std::string;
+    using ReturnType = DataTypeJsonb;
+    using ColumnType = ColumnJsonb;
+    static const bool only_check_exists = false;
+    static const bool only_get_type = false;
+};
+
+struct JsonbTypeType {
+    using T = std::string;
+    using ReturnType = DataTypeString;
+    using ColumnType = ColumnString;
+    static const bool only_check_exists = false;
+    static const bool only_get_type = true;
+};
+
+struct JsonbExists : public JsonbExtractImpl<JsonbTypeExists> {
+    static constexpr auto name = "jsonb_exists_path";
+};
+
+struct JsonbExtractIsnull : public JsonbExtractImpl<JsonbTypeNull> {
+    static constexpr auto name = "jsonb_extract_isnull";
+};
+
+struct JsonbExtractBool : public JsonbExtractImpl<JsonbTypeBool> {
+    static constexpr auto name = "jsonb_extract_bool";
+};
+
+struct JsonbExtractInt : public JsonbExtractImpl<JsonbTypeInt> {
+    static constexpr auto name = "jsonb_extract_int";
+};
+
+struct JsonbExtractBigInt : public JsonbExtractImpl<JsonbTypeInt64> {
+    static constexpr auto name = "jsonb_extract_bigint";
+};
+
+struct JsonbExtractDouble : public JsonbExtractImpl<JsonbTypeDouble> {
+    static constexpr auto name = "jsonb_extract_double";
+};
+
+struct JsonbExtractString : public JsonbExtractStringImpl<JsonbTypeString> {
+    static constexpr auto name = "jsonb_extract_string";
+};
+
+struct JsonbExtractJsonb : public JsonbExtractStringImpl<JsonbTypeJson> {
+    static constexpr auto name = "jsonb_extract";
+};
+
+struct JsonbType : public JsonbExtractStringImpl<JsonbTypeType> {
+    static constexpr auto name = "jsonb_type";
+};
+
+using FunctionJsonbExists = FunctionJsonbExtract<JsonbExists>;
+using FunctionJsonbType = FunctionJsonbExtract<JsonbType>;
+
+using FunctionJsonbExtractIsnull = FunctionJsonbExtract<JsonbExtractIsnull>;
+using FunctionJsonbExtractBool = FunctionJsonbExtract<JsonbExtractBool>;
+using FunctionJsonbExtractInt = FunctionJsonbExtract<JsonbExtractInt>;
+using FunctionJsonbExtractBigInt = FunctionJsonbExtract<JsonbExtractBigInt>;
+using FunctionJsonbExtractDouble = FunctionJsonbExtract<JsonbExtractDouble>;
+using FunctionJsonbExtractString = FunctionJsonbExtract<JsonbExtractString>;
+using FunctionJsonbExtractJsonb = FunctionJsonbExtract<JsonbExtractJsonb>;
+
 void register_function_jsonb(SimpleFunctionFactory& factory) {
     factory.register_function<FunctionJsonbParse>("jsonb_parse");
     factory.register_function<FunctionJsonbParseErrorNull>("jsonb_parse_error_to_null");
@@ -309,6 +659,17 @@ void register_function_jsonb(SimpleFunctionFactory& factory) {
             "jsonb_parse_notnull_error_to_value");
     factory.register_function<FunctionJsonbParseNotnullErrorInvalid>(
             "jsonb_parse_notnull_error_to_invalid");
+
+    factory.register_function<FunctionJsonbExists>();
+    factory.register_function<FunctionJsonbType>();
+
+    factory.register_function<FunctionJsonbExtractIsnull>();
+    factory.register_function<FunctionJsonbExtractBool>();
+    factory.register_function<FunctionJsonbExtractInt>();
+    factory.register_function<FunctionJsonbExtractBigInt>();
+    factory.register_function<FunctionJsonbExtractDouble>();
+    factory.register_function<FunctionJsonbExtractString>();
+    factory.register_function<FunctionJsonbExtractJsonb>();
 }
 
 } // namespace doris::vectorized
diff --git a/be/test/vec/function/function_jsonb_test.cpp b/be/test/vec/function/function_jsonb_test.cpp
index 68f58e9bc4..a31bcc8939 100644
--- a/be/test/vec/function/function_jsonb_test.cpp
+++ b/be/test/vec/function/function_jsonb_test.cpp
@@ -35,7 +35,7 @@ TEST(FunctionJsonbTEST, JsonbParseTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -56,37 +56,37 @@ TEST(FunctionJsonbTEST, JsonbParseTest) {
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 }
 
@@ -100,7 +100,7 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToNullTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -135,7 +135,7 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToValueTest) {
             {{STRING("false"), STRING("{}")}, STRING("false")},
             {{STRING("100"), STRING("{}")}, STRING("100")},                                 //int8
             {{STRING("10000"), STRING("{}")}, STRING("10000")},                             // int16
-            {{STRING("1073741820"), STRING("{}")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000"), STRING("{}")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976"), STRING("{}")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18"), STRING("{}")}, STRING("6.18")},           // double
             {{STRING(R"("abcd")"), STRING("{}")}, STRING(R"("abcd")")}, // string
@@ -173,7 +173,7 @@ TEST(FunctionJsonbTEST, JsonbParseErrorToInvalidTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -208,7 +208,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -229,37 +229,37 @@ TEST(FunctionJsonbTEST, JsonbParseNullableTest) {
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 }
 
@@ -273,7 +273,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToNullTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -308,7 +308,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToValueTest) {
             {{STRING("false"), STRING("{}")}, STRING("false")},
             {{STRING("100"), STRING("{}")}, STRING("100")},                                 //int8
             {{STRING("10000"), STRING("{}")}, STRING("10000")},                             // int16
-            {{STRING("1073741820"), STRING("{}")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000"), STRING("{}")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976"), STRING("{}")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18"), STRING("{}")}, STRING("6.18")},           // double
             {{STRING(R"("abcd")"), STRING("{}")}, STRING(R"("abcd")")}, // string
@@ -346,7 +346,7 @@ TEST(FunctionJsonbTEST, JsonbParseNullableErrorToInvalidTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -381,7 +381,7 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -402,37 +402,37 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullTest) {
     DataSet data_set_invalid = {
             {{STRING("abc")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("'abc'")}, Null()}, // invalid string
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("100x")}, Null()}, // invalid int
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("6.a8")}, Null()}, // invalid double
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("{x")}, Null()}, // invalid object
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 
     data_set_invalid = {
             {{STRING("[123, abc]")}, Null()} // invalid array
     };
-    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid);
+    st = check_function<DataTypeJsonb, false>(func_name, input_types, data_set_invalid, true);
     EXPECT_NE(Status::OK(), st);
 }
 
@@ -446,7 +446,7 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToValueTest) {
             {{STRING("false"), STRING("{}")}, STRING("false")},
             {{STRING("100"), STRING("{}")}, STRING("100")},                                 //int8
             {{STRING("10000"), STRING("{}")}, STRING("10000")},                             // int16
-            {{STRING("1073741820"), STRING("{}")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000"), STRING("{}")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976"), STRING("{}")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18"), STRING("{}")}, STRING("6.18")},           // double
             {{STRING(R"("abcd")"), STRING("{}")}, STRING(R"("abcd")")}, // string
@@ -484,7 +484,7 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToInvalidTest) {
             {{STRING("false")}, STRING("false")},
             {{STRING("100")}, STRING("100")},                                 //int8
             {{STRING("10000")}, STRING("10000")},                             // int16
-            {{STRING("1073741820")}, STRING("1073741820")},                   // int32
+            {{STRING("1000000000")}, STRING("1000000000")},                   // int32
             {{STRING("1152921504606846976")}, STRING("1152921504606846976")}, // int64
             {{STRING("6.18")}, STRING("6.18")},                               // double
             {{STRING(R"("abcd")")}, STRING(R"("abcd")")},                     // string
@@ -509,4 +509,717 @@ TEST(FunctionJsonbTEST, JsonbParseNotnullErrorToInvalidTest) {
     EXPECT_EQ(Status::OK(), st);
 }
 
+TEST(FunctionJsonbTEST, JsonbExtractTest) {
+    std::string func_name = "jsonb_extract";
+    InputTypeSet input_types = {TypeIndex::JSONB, TypeIndex::String};
+
+    // jsonb_extract root
+    DataSet data_set = {
+            {{STRING("null"), STRING("$")}, STRING("null")},
+            {{STRING("true"), STRING("$")}, STRING("true")},
+            {{STRING("false"), STRING("$")}, STRING("false")},
+            {{STRING("100"), STRING("$")}, STRING("100")},                                 //int8
+            {{STRING("10000"), STRING("$")}, STRING("10000")},                             // int16
+            {{STRING("1000000000"), STRING("$")}, STRING("1000000000")},                   // int32
+            {{STRING("1152921504606846976"), STRING("$")}, STRING("1152921504606846976")}, // int64
+            {{STRING("6.18"), STRING("$")}, STRING("6.18")},                               // double
+            {{STRING(R"("abcd")"), STRING("$")}, STRING(R"("abcd")")},                     // string
+            {{STRING("{}"), STRING("$")}, STRING("{}")}, // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$")},
+             STRING(R"({"k1":"v31","k2":300})")},                       // object
+            {{STRING("[]"), STRING("$")}, STRING("[]")},                // empty array
+            {{STRING("[123, 456]"), STRING("$")}, STRING("[123,456]")}, // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$")},
+             STRING(R"(["abc","def"])")}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), 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(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);
+
+    // jsonb_extract obejct
+    data_set = {
+            {{STRING("null"), STRING("$.k1")}, Null()},
+            {{STRING("true"), STRING("$.k1")}, Null()},
+            {{STRING("false"), STRING("$.k1")}, Null()},
+            {{STRING("100"), STRING("$.k1")}, Null()},                 //int8
+            {{STRING("10000"), STRING("$.k1")}, Null()},               // int16
+            {{STRING("1000000000"), STRING("$.k1")}, Null()},          // int32
+            {{STRING("1152921504606846976"), STRING("$.k1")}, Null()}, // int64
+            {{STRING("6.18"), STRING("$.k1")}, Null()},                // double
+            {{STRING(R"("abcd")"), STRING("$.k1")}, Null()},           // string
+            {{STRING("{}"), STRING("$.k1")}, Null()},                  // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$.k1")}, STRING(R"("v31")")}, // object
+            {{STRING("[]"), STRING("$.k1")}, Null()},                // empty array
+            {{STRING("[123, 456]"), STRING("$.k1")}, Null()},        // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$.k1")}, Null()}, // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$.k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$.k1")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract array
+    data_set = {
+            {{STRING("null"), STRING("$[0]")}, Null()},
+            {{STRING("true"), STRING("$[0]")}, Null()},
+            {{STRING("false"), STRING("$[0]")}, Null()},
+            {{STRING("100"), STRING("$[0]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0]")}, Null()},                         // empty array
+            {{STRING("null"), STRING("$[1]")}, Null()},
+            {{STRING("true"), STRING("$[1]")}, Null()},
+            {{STRING("false"), STRING("$[1]")}, Null()},
+            {{STRING("100"), STRING("$[1]")}, Null()},                           //int8
+            {{STRING("10000"), STRING("$[1]")}, Null()},                         // int16
+            {{STRING("1000000000"), STRING("$[1]")}, Null()},                    // int32
+            {{STRING("1152921504606846976"), STRING("$[1]")}, Null()},           // int64
+            {{STRING("6.18"), STRING("$[1]")}, Null()},                          // double
+            {{STRING(R"("abcd")"), STRING("$[1]")}, Null()},                     // string
+            {{STRING("{}"), STRING("$[1]")}, Null()},                            // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[1]")}, Null()},    // object
+            {{STRING("[]"), STRING("$[1]")}, Null()},                            // empty array
+            {{STRING("[123, 456]"), STRING("$[0]")}, STRING("123")},             // int array
+            {{STRING("[123, 456]"), STRING("$[1]")}, STRING("456")},             // int array
+            {{STRING("[123, 456]"), STRING("$[2]")}, Null()},                    // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0]")}, STRING(R"("abc")")}, // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[1]")}, STRING(R"("def")")}, // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[2]")}, Null()},             // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0]")},
+             STRING("null")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[1]")},
+             STRING("true")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[2]")},
+             STRING("false")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[3]")},
+             STRING("100")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[4]")},
+             STRING("6.18")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[5]")},
+             STRING(R"("abc")")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[6]")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0]")},
+             STRING(R"({"k1":"v41","k2":400})")}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[1]")},
+             STRING("1")}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[2]")},
+             STRING(R"("a")")}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[3]")},
+             STRING("3.14")}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[4]")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract $[0].k1
+    data_set = {
+            {{STRING("null"), STRING("$[0].k1")}, Null()},
+            {{STRING("true"), STRING("$[0].k1")}, Null()},
+            {{STRING("false"), STRING("$[0].k1")}, Null()},
+            {{STRING("100"), STRING("$[0].k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0].k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0].k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0].k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0].k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0].k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0].k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0].k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0].k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0].k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0].k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0].k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k1")},
+             STRING(R"("v41")")}, // complex array
+    };
+
+    st = check_function<DataTypeJsonb, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+}
+
+TEST(FunctionJsonbTEST, JsonbExtractStringTest) {
+    std::string func_name = "jsonb_extract_string";
+    InputTypeSet input_types = {TypeIndex::JSONB, TypeIndex::String};
+
+    // jsonb_extract root
+    DataSet data_set = {
+            {{STRING("null"), STRING("$")}, Null()},
+            {{STRING("true"), STRING("$")}, Null()},
+            {{STRING("false"), STRING("$")}, Null()},
+            {{STRING("100"), STRING("$")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$")}, STRING("abcd")},          // string
+            {{STRING("{}"), STRING("$")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$")}, Null()}, // object
+            {{STRING("[]"), STRING("$")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$")},
+             Null()}, // complex array
+    };
+
+    auto st = check_function<DataTypeString, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract obejct
+    data_set = {
+            {{STRING("null"), STRING("$.k1")}, Null()},
+            {{STRING("true"), STRING("$.k1")}, Null()},
+            {{STRING("false"), STRING("$.k1")}, Null()},
+            {{STRING("100"), STRING("$.k1")}, Null()},                               //int8
+            {{STRING("10000"), STRING("$.k1")}, Null()},                             // int16
+            {{STRING("1000000000"), STRING("$.k1")}, Null()},                        // int32
+            {{STRING("1152921504606846976"), STRING("$.k1")}, Null()},               // int64
+            {{STRING("6.18"), STRING("$.k1")}, Null()},                              // double
+            {{STRING(R"("abcd")"), STRING("$.k1")}, Null()},                         // string
+            {{STRING("{}"), STRING("$.k1")}, Null()},                                // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$.k1")}, STRING("v31")}, // object
+            {{STRING("[]"), STRING("$.k1")}, Null()},                                // empty array
+            {{STRING("[123, 456]"), STRING("$.k1")}, Null()},                        // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$.k1")}, Null()},                 // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$.k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$.k1")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract array
+    data_set = {
+            {{STRING("null"), STRING("$[0]")}, Null()},
+            {{STRING("true"), STRING("$[0]")}, Null()},
+            {{STRING("false"), STRING("$[0]")}, Null()},
+            {{STRING("100"), STRING("$[0]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0]")}, Null()},                         // empty array
+            {{STRING("null"), STRING("$[1]")}, Null()},
+            {{STRING("true"), STRING("$[1]")}, Null()},
+            {{STRING("false"), STRING("$[1]")}, Null()},
+            {{STRING("100"), STRING("$[1]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[1]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[1]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[1]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[1]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[1]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[1]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[1]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[1]")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0]")}, Null()},                 // int array
+            {{STRING("[123, 456]"), STRING("$[1]")}, Null()},                 // int array
+            {{STRING("[123, 456]"), STRING("$[2]")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0]")}, STRING("abc")},   // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[1]")}, STRING("def")},   // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[2]")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[1]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[2]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[3]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[4]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[5]")},
+             STRING("abc")}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[6]")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[1]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[2]")},
+             STRING("a")}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[3]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[4]")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract $[0].k1
+    data_set = {
+            {{STRING("null"), STRING("$[0].k1")}, Null()},
+            {{STRING("true"), STRING("$[0].k1")}, Null()},
+            {{STRING("false"), STRING("$[0].k1")}, Null()},
+            {{STRING("100"), STRING("$[0].k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0].k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0].k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0].k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0].k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0].k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0].k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0].k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0].k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0].k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0].k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0].k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k1")},
+             STRING("v41")}, // complex array
+    };
+
+    st = check_function<DataTypeString, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+}
+
+TEST(FunctionJsonbTEST, JsonbExtractIntTest) {
+    std::string func_name = "jsonb_extract_int";
+    InputTypeSet input_types = {TypeIndex::JSONB, TypeIndex::String};
+
+    // jsonb_extract root
+    DataSet data_set = {
+            {{STRING("null"), STRING("$")}, Null()},
+            {{STRING("true"), STRING("$")}, Null()},
+            {{STRING("false"), STRING("$")}, Null()},
+            {{STRING("100"), STRING("$")}, INT(100)},                      //int8
+            {{STRING("10000"), STRING("$")}, INT(10000)},                  // int16
+            {{STRING("1000000000"), STRING("$")}, INT(1000000000)},        // int32
+            {{STRING("1152921504606846976"), STRING("$")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$")}, Null()},                  // string
+            {{STRING("{}"), STRING("$")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$")}, Null()}, // object
+            {{STRING("[]"), STRING("$")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$")},
+             Null()}, // complex array
+    };
+
+    auto st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract obejct
+    data_set = {
+            {{STRING("null"), STRING("$.k1")}, Null()},
+            {{STRING("true"), STRING("$.k1")}, Null()},
+            {{STRING("false"), STRING("$.k1")}, Null()},
+            {{STRING("100"), STRING("$.k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$.k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$.k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$.k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$.k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$.k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$.k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$.k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$.k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$.k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$.k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$.k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$.k1")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract array
+    data_set = {
+            {{STRING("null"), STRING("$[0]")}, Null()},
+            {{STRING("true"), STRING("$[0]")}, Null()},
+            {{STRING("false"), STRING("$[0]")}, Null()},
+            {{STRING("100"), STRING("$[0]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0]")}, Null()},                         // empty array
+            {{STRING("null"), STRING("$[1]")}, Null()},
+            {{STRING("true"), STRING("$[1]")}, Null()},
+            {{STRING("false"), STRING("$[1]")}, Null()},
+            {{STRING("100"), STRING("$[1]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[1]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[1]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[1]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[1]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[1]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[1]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[1]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[1]")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0]")}, INT(123)},               // int array
+            {{STRING("[123, 456]"), STRING("$[1]")}, INT(456)},               // int array
+            {{STRING("[123, 456]"), STRING("$[2]")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[1]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[2]")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[1]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[2]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[3]")},
+             INT(100)}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[4]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[5]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[6]")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[1]")},
+             INT(1)}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[2]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[3]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[4]")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract $[0].k1
+    data_set = {
+            {{STRING("null"), STRING("$[0].k1")}, Null()},
+            {{STRING("true"), STRING("$[0].k1")}, Null()},
+            {{STRING("false"), STRING("$[0].k1")}, Null()},
+            {{STRING("100"), STRING("$[0].k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0].k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0].k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0].k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0].k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0].k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0].k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0].k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0].k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0].k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0].k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0].k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k1")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k2")},
+             INT(400)}, // complex array
+    };
+
+    st = check_function<DataTypeInt32, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+}
+
+TEST(FunctionJsonbTEST, JsonbExtractBigIntTest) {
+    std::string func_name = "jsonb_extract_bigint";
+    InputTypeSet input_types = {TypeIndex::JSONB, TypeIndex::String};
+
+    // jsonb_extract root
+    DataSet data_set = {
+            {{STRING("null"), STRING("$")}, Null()},
+            {{STRING("true"), STRING("$")}, Null()},
+            {{STRING("false"), STRING("$")}, Null()},
+            {{STRING("100"), STRING("$")}, BIGINT(100)},                                 //int8
+            {{STRING("10000"), STRING("$")}, BIGINT(10000)},                             // int16
+            {{STRING("1000000000"), STRING("$")}, BIGINT(1000000000)},                   // int32
+            {{STRING("1152921504606846976"), STRING("$")}, BIGINT(1152921504606846976)}, // int64
+            {{STRING("6.18"), STRING("$")}, Null()},                                     // double
+            {{STRING(R"("abcd")"), STRING("$")}, Null()},                                // string
+            {{STRING("{}"), STRING("$")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$")}, Null()}, // object
+            {{STRING("[]"), STRING("$")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$")},
+             Null()}, // complex array
+    };
+
+    auto st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract obejct
+    data_set = {
+            {{STRING("null"), STRING("$.k1")}, Null()},
+            {{STRING("true"), STRING("$.k1")}, Null()},
+            {{STRING("false"), STRING("$.k1")}, Null()},
+            {{STRING("100"), STRING("$.k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$.k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$.k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$.k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$.k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$.k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$.k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$.k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$.k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$.k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$.k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$.k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$.k1")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract array
+    data_set = {
+            {{STRING("null"), STRING("$[0]")}, Null()},
+            {{STRING("true"), STRING("$[0]")}, Null()},
+            {{STRING("false"), STRING("$[0]")}, Null()},
+            {{STRING("100"), STRING("$[0]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0]")}, Null()},                         // empty array
+            {{STRING("null"), STRING("$[1]")}, Null()},
+            {{STRING("true"), STRING("$[1]")}, Null()},
+            {{STRING("false"), STRING("$[1]")}, Null()},
+            {{STRING("100"), STRING("$[1]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[1]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[1]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[1]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[1]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[1]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[1]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[1]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[1]")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0]")}, BIGINT(123)},            // int array
+            {{STRING("[123, 456]"), STRING("$[1]")}, BIGINT(456)},            // int array
+            {{STRING("[123, 456]"), STRING("$[2]")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[1]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[2]")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[1]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[2]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[3]")},
+             BIGINT(100)}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[4]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[5]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[6]")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[1]")},
+             BIGINT(1)}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[2]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[3]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[4]")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract $[0].k1
+    data_set = {
+            {{STRING("null"), STRING("$[0].k1")}, Null()},
+            {{STRING("true"), STRING("$[0].k1")}, Null()},
+            {{STRING("false"), STRING("$[0].k1")}, Null()},
+            {{STRING("100"), STRING("$[0].k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0].k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0].k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0].k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0].k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0].k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0].k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0].k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0].k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0].k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0].k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0].k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k1")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k2")},
+             BIGINT(400)}, // complex array
+    };
+
+    st = check_function<DataTypeInt64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+}
+
+TEST(FunctionJsonbTEST, JsonbExtractDoubleTest) {
+    std::string func_name = "jsonb_extract_double";
+    InputTypeSet input_types = {TypeIndex::JSONB, TypeIndex::String};
+
+    // jsonb_extract root
+    DataSet data_set = {
+            {{STRING("null"), STRING("$")}, Null()},
+            {{STRING("true"), STRING("$")}, Null()},
+            {{STRING("false"), STRING("$")}, Null()},
+            {{STRING("100"), STRING("$")}, DOUBLE(100)},                                 //int8
+            {{STRING("10000"), STRING("$")}, DOUBLE(10000)},                             // int16
+            {{STRING("1000000000"), STRING("$")}, DOUBLE(1000000000)},                   // int32
+            {{STRING("1152921504606846976"), STRING("$")}, DOUBLE(1152921504606846976)}, // int64
+            {{STRING("6.18"), STRING("$")}, DOUBLE(6.18)},                               // double
+            {{STRING(R"("abcd")"), STRING("$")}, Null()},                                // string
+            {{STRING("{}"), STRING("$")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$")}, Null()}, // object
+            {{STRING("[]"), STRING("$")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$")},
+             Null()}, // complex array
+    };
+
+    auto st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract obejct
+    data_set = {
+            {{STRING("null"), STRING("$.k1")}, Null()},
+            {{STRING("true"), STRING("$.k1")}, Null()},
+            {{STRING("false"), STRING("$.k1")}, Null()},
+            {{STRING("100"), STRING("$.k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$.k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$.k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$.k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$.k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$.k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$.k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$.k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$.k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$.k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$.k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$.k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$.k1")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract array
+    data_set = {
+            {{STRING("null"), STRING("$[0]")}, Null()},
+            {{STRING("true"), STRING("$[0]")}, Null()},
+            {{STRING("false"), STRING("$[0]")}, Null()},
+            {{STRING("100"), STRING("$[0]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0]")}, Null()},                         // empty array
+            {{STRING("null"), STRING("$[1]")}, Null()},
+            {{STRING("true"), STRING("$[1]")}, Null()},
+            {{STRING("false"), STRING("$[1]")}, Null()},
+            {{STRING("100"), STRING("$[1]")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[1]")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[1]")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[1]")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[1]")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[1]")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[1]")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[1]")}, Null()}, // object
+            {{STRING("[]"), STRING("$[1]")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0]")}, DOUBLE(123)},            // int array
+            {{STRING("[123, 456]"), STRING("$[1]")}, DOUBLE(456)},            // int array
+            {{STRING("[123, 456]"), STRING("$[2]")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[1]")}, Null()},          // string array
+            {{STRING(R"(["abc", "def"])"), STRING("$[2]")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[1]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[2]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[3]")},
+             DOUBLE(100)}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[4]")},
+             DOUBLE(6.18)}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[5]")},
+             Null()}, // multi type array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[6]")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[1]")},
+             DOUBLE(1)}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[2]")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[3]")},
+             DOUBLE(3.14)}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[4]")},
+             Null()}, // complex array
+    };
+
+    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+
+    // jsonb_extract $[0].k1
+    data_set = {
+            {{STRING("null"), STRING("$[0].k1")}, Null()},
+            {{STRING("true"), STRING("$[0].k1")}, Null()},
+            {{STRING("false"), STRING("$[0].k1")}, Null()},
+            {{STRING("100"), STRING("$[0].k1")}, Null()},                        //int8
+            {{STRING("10000"), STRING("$[0].k1")}, Null()},                      // int16
+            {{STRING("1000000000"), STRING("$[0].k1")}, Null()},                 // int32
+            {{STRING("1152921504606846976"), STRING("$[0].k1")}, Null()},        // int64
+            {{STRING("6.18"), STRING("$[0].k1")}, Null()},                       // double
+            {{STRING(R"("abcd")"), STRING("$[0].k1")}, Null()},                  // string
+            {{STRING("{}"), STRING("$[0].k1")}, Null()},                         // empty object
+            {{STRING(R"({"k1":"v31", "k2": 300})"), STRING("$[0].k1")}, Null()}, // object
+            {{STRING("[]"), STRING("$[0].k1")}, Null()},                         // empty array
+            {{STRING("[123, 456]"), STRING("$[0].k1")}, Null()},                 // int array
+            {{STRING(R"(["abc", "def"])"), STRING("$[0].k1")}, Null()},          // string array
+            {{STRING(R"([null, true, false, 100, 6.18, "abc"])"), STRING("$[0].k1")},
+             Null()}, // multi type array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k1")},
+             Null()}, // complex array
+            {{STRING(R"([{"k1":"v41", "k2": 400}, 1, "a", 3.14])"), STRING("$[0].k2")},
+             DOUBLE(400)}, // complex array
+    };
+
+    st = check_function<DataTypeFloat64, true>(func_name, input_types, data_set);
+    EXPECT_EQ(Status::OK(), st);
+}
+
 } // namespace doris::vectorized
diff --git a/be/test/vec/function/function_test_util.cpp b/be/test/vec/function/function_test_util.cpp
index 3cc25ccd95..92febd0640 100644
--- a/be/test/vec/function/function_test_util.cpp
+++ b/be/test/vec/function/function_test_util.cpp
@@ -17,6 +17,7 @@
 
 #include "vec/function/function_test_util.h"
 
+#include "runtime/jsonb_value.h"
 #include "vec/data_types/data_type_array.h"
 #include "vec/data_types/data_type_bitmap.h"
 #include "vec/data_types/data_type_decimal.h"
@@ -66,6 +67,10 @@ size_t type_index_to_data_type(const std::vector<std::any>& input_types, size_t
         desc.type = doris_udf::FunctionContext::TYPE_STRING;
         type = std::make_shared<DataTypeString>();
         return 1;
+    case TypeIndex::JSONB:
+        desc.type = doris_udf::FunctionContext::TYPE_JSONB;
+        type = std::make_shared<DataTypeJsonb>();
+        return 1;
     case TypeIndex::BitMap:
         desc.type = doris_udf::FunctionContext::TYPE_OBJECT;
         type = std::make_shared<DataTypeBitMap>();
@@ -177,6 +182,10 @@ bool insert_cell(MutableColumnPtr& column, DataTypePtr type_ptr, const std::any&
     if (type.is_string()) {
         auto str = std::any_cast<ut_type::STRING>(cell);
         column->insert_data(str.c_str(), str.size());
+    } else if (type.is_json()) {
+        auto str = std::any_cast<ut_type::STRING>(cell);
+        JsonBinaryValue jsonb_val(str.c_str(), str.size());
+        column->insert_data(jsonb_val.value(), jsonb_val.size());
     } else if (type.idx == TypeIndex::BitMap) {
         BitmapValue* bitmap = std::any_cast<BitmapValue*>(cell);
         column->insert_data((char*)bitmap, sizeof(BitmapValue));
diff --git a/be/test/vec/function/function_test_util.h b/be/test/vec/function/function_test_util.h
index 1d4d56f029..5d652d9b6c 100644
--- a/be/test/vec/function/function_test_util.h
+++ b/be/test/vec/function/function_test_util.h
@@ -163,7 +163,7 @@ void check_vec_table_function(TableFunction* fn, const InputTypeSet& input_types
 // A DataSet with a constant column can only have one row of data
 template <typename ReturnType, bool nullable = false>
 Status check_function(const std::string& func_name, const InputTypeSet& input_types,
-                      const DataSet& data_set) {
+                      const DataSet& data_set, bool expect_fail = false) {
     // 1.0 create data type
     ut_type::UTDataTypeDescs descs;
     EXPECT_TRUE(parse_ut_data_type(input_types, descs));
@@ -234,16 +234,20 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty
     FunctionUtils fn_utils(fn_ctx_return, arg_types, 0);
     auto* fn_ctx = fn_utils.get_fn_ctx();
     fn_ctx->impl()->set_constant_cols(constant_cols);
-    RETURN_IF_ERROR(func->prepare(fn_ctx, FunctionContext::FRAGMENT_LOCAL));
-    RETURN_IF_ERROR(func->prepare(fn_ctx, FunctionContext::THREAD_LOCAL));
+    func->prepare(fn_ctx, FunctionContext::FRAGMENT_LOCAL);
+    func->prepare(fn_ctx, FunctionContext::THREAD_LOCAL);
 
     block.insert({nullptr, return_type, "result"});
 
     auto result = block.columns() - 1;
-    RETURN_IF_ERROR(func->execute(fn_ctx, block, arguments, result, row_size));
+    if (expect_fail) {
+        RETURN_IF_ERROR(func->execute(fn_ctx, block, arguments, result, row_size));
+    } else {
+        func->execute(fn_ctx, block, arguments, result, row_size);
+    }
 
-    RETURN_IF_ERROR(func->close(fn_ctx, FunctionContext::THREAD_LOCAL));
-    RETURN_IF_ERROR(func->close(fn_ctx, FunctionContext::FRAGMENT_LOCAL));
+    func->close(fn_ctx, FunctionContext::THREAD_LOCAL);
+    func->close(fn_ctx, FunctionContext::FRAGMENT_LOCAL);
 
     // 3. check the result of function
     ColumnPtr column = block.get_columns()[result];
@@ -256,13 +260,13 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty
                 auto s = column->get_data_at(i);
                 if (expect_data.size() == 0) {
                     // zero size result means invalid
-                    EXPECT_EQ(0, s.size) << " invalid result size should be 0 for row " << i;
+                    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) << " for row " << i;
+                    EXPECT_EQ(to_json.jsonb_to_string(val), expect_data) << " at row " << i;
                 }
             } else {
                 Field field;
@@ -273,20 +277,20 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty
 
                 if constexpr (std::is_same_v<ReturnType, DataTypeDecimal<Decimal128>>) {
                     const auto& column_data = field.get<DecimalField<Decimal128>>().get_value();
-                    EXPECT_EQ(expect_data.value, column_data.value);
+                    EXPECT_EQ(expect_data.value, column_data.value) << " at row " << i;
                 } else if constexpr (std::is_same_v<ReturnType, DataTypeFloat32>) {
                     const auto& column_data = field.get<DataTypeFloat64::FieldType>();
-                    EXPECT_EQ(expect_data, column_data);
+                    EXPECT_EQ(expect_data, column_data) << " at row " << i;
                 } else {
                     const auto& column_data = field.get<typename ReturnType::FieldType>();
-                    EXPECT_EQ(expect_data, column_data);
+                    EXPECT_EQ(expect_data, column_data) << " at row " << i;
                 }
             }
         };
 
         if constexpr (nullable) {
             bool is_null = data_set[i].second.type() == typeid(Null);
-            EXPECT_EQ(is_null, column->is_null_at(i));
+            EXPECT_EQ(is_null, column->is_null_at(i)) << " at row " << i;
             if (!is_null) check_column_data();
         } else {
             check_column_data();
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index b97dbacd68..f1bb4faf86 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -2298,6 +2298,62 @@ visible_functions = [
         'fake_symble_for_no_vec', '', '',
         'vec', ''],
 
+    [['jsonb_exists_path'], 'BOOLEAN', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', ''],
+    [['jsonb_exists_path'], 'BOOLEAN', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', ''],
+    [['jsonb_type'], 'STRING', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_type'], 'STRING', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+
+    [['jsonb_extract'], 'JSONB', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract'], 'JSONB', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_isnull'], 'BOOLEAN', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_isnull'], 'BOOLEAN', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_bool'], 'BOOLEAN', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_bool'], 'BOOLEAN', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_int'], 'INT', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_int'], 'INT', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_bigint'], 'BIGINT', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_bigint'], 'BIGINT', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_double'], 'DOUBLE', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_double'], 'DOUBLE', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_string'], 'STRING', ['JSONB', 'VARCHAR'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+    [['jsonb_extract_string'], 'STRING', ['JSONB', 'STRING'],
+        'fake_symble_for_no_vec', '', '',
+        'vec', 'ALWAYS_NULLABLE'],
+
     # Json functions
     [['get_json_int'], 'INT', ['VARCHAR', 'VARCHAR'],
         '_ZN5doris13JsonFunctions12get_json_intEPN9doris_udf15FunctionContextERKNS1_9StringValES6_',


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