diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp index 425145d2408086..8fe21f5df9fe6f 100644 --- a/be/src/vec/functions/function_jsonb.cpp +++ b/be/src/vec/functions/function_jsonb.cpp @@ -431,6 +431,177 @@ class FunctionJsonbExtract : public IFunction { } }; +class FunctionJsonbKeys : public IFunction { +public: + static constexpr auto name = "json_keys"; + static constexpr auto alias = "jsonb_keys"; + static FunctionPtr create() { return std::make_shared(); } + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { + return make_nullable( + std::make_shared(make_nullable(std::make_shared()))); + } + + Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count) const override { + DCHECK_GE(arguments.size(), 1); + if (arguments.size() != 1 && arguments.size() != 2) { + // here has argument param error + return Status::InvalidArgument("json_keys should have 1 or 2 arguments"); + } + + ColumnPtr jsonb_data_column = nullptr; + const NullMap* data_null_map = nullptr; + // prepare jsonb data column + jsonb_data_column = unpack_if_const(block.get_by_position(arguments[0]).column).first; + if (block.get_by_position(arguments[0]).column->is_nullable()) { + const auto* nullable = check_and_get_column(jsonb_data_column); + jsonb_data_column = nullable->get_nested_column_ptr(); + data_null_map = &nullable->get_null_map_data(); + } + const ColumnString* col_from_string = check_and_get_column(jsonb_data_column); + + // prepare parse path column prepare, maybe we do not have path column + ColumnPtr jsonb_path_column = nullptr; + const ColumnString* jsonb_path_col = nullptr; + bool path_const = false; + const NullMap* path_null_map = nullptr; + if (arguments.size() == 2) { + // we have should have a ColumnString for path + std::tie(jsonb_path_column, path_const) = + unpack_if_const(block.get_by_position(arguments[1]).column); + if (block.get_by_position(arguments[1]).column->is_nullable()) { + const auto* nullable = check_and_get_column(jsonb_path_column); + jsonb_path_column = nullable->get_nested_column_ptr(); + path_null_map = &nullable->get_null_map_data(); + } + jsonb_path_col = check_and_get_column(jsonb_path_column); + } + + auto null_map = ColumnUInt8::create(input_rows_count, 0); + NullMap& res_null_map = null_map->get_data(); + + auto dst_arr = ColumnArray::create( + ColumnNullable::create(ColumnString::create(), ColumnUInt8::create()), + ColumnArray::ColumnOffsets::create()); + ColumnNullable& dst_nested_column = assert_cast(dst_arr->get_data()); + + Status st; + if (jsonb_path_column) { + if (path_const) { + st = inner_loop_impl(input_rows_count, *dst_arr, dst_nested_column, + res_null_map, *col_from_string, data_null_map, + jsonb_path_col, path_null_map); + } else { + st = inner_loop_impl(input_rows_count, *dst_arr, dst_nested_column, + res_null_map, *col_from_string, data_null_map, + jsonb_path_col, path_null_map); + } + } else { + st = inner_loop_impl(input_rows_count, *dst_arr, dst_nested_column, + res_null_map, *col_from_string, data_null_map, + jsonb_path_col, path_null_map); + } + if (!st.ok()) { + return st; + } + block.get_by_position(result).column = + ColumnNullable::create(std::move(dst_arr), std::move(null_map)); + return st; + } + +private: + template + static ALWAYS_INLINE Status inner_loop_impl(size_t input_rows_count, ColumnArray& dst_arr, + ColumnNullable& dst_nested_column, + NullMap& res_null_map, + const ColumnString& col_from_string, + const NullMap* jsonb_data_nullmap, + const ColumnString* jsonb_path_column, + const NullMap* path_null_map) { + // if path is const, we just need to parse it once + JsonbPath const_path; + if constexpr (JSONB_PATH_PARAM && JSON_PATH_CONST) { + StringRef r_raw_ref = jsonb_path_column->get_data_at(0); + if (!const_path.seek(r_raw_ref.data, r_raw_ref.size)) { + return Status::InvalidArgument( + "Json path error: {} for value: {}", + JsonbErrMsg::getErrMsg(JsonbErrType::E_INVALID_JSON_PATH), + r_raw_ref.to_string()); + } + } + const auto& ldata = col_from_string.get_chars(); + const auto& loffsets = col_from_string.get_offsets(); + for (size_t i = 0; i < input_rows_count; ++i) { + // if jsonb data is null or path column is null , we should return null + if (jsonb_data_nullmap && (&jsonb_data_nullmap)[i]) { + res_null_map[i] = 1; + dst_arr.insert_default(); + continue; + } + if constexpr (JSONB_PATH_PARAM && !JSON_PATH_CONST) { + if (path_null_map && (&path_null_map)[i]) { + res_null_map[i] = 1; + dst_arr.insert_default(); + continue; + } + } + // extract jsonb keys + size_t l_off = loffsets[i - 1]; + size_t l_size = loffsets[i] - l_off; + if (l_size == 0) { + res_null_map[i] = 1; + dst_arr.insert_default(); + continue; + } + const char* l_raw = reinterpret_cast(&ldata[l_off]); + JsonbDocument* doc = JsonbDocument::createDocument(l_raw, l_size); + if (UNLIKELY(!doc || !doc->getValue())) { + dst_arr.clear(); + return Status::InvalidArgument("jsonb data is invalid"); + } + JsonbValue* obj_val; + if constexpr (JSONB_PATH_PARAM) { + if constexpr (!JSON_PATH_CONST) { + const ColumnString::Chars& rdata = jsonb_path_column->get_chars(); + const ColumnString::Offsets& roffsets = jsonb_path_column->get_offsets(); + size_t r_off = roffsets[i - 1]; + size_t r_size = roffsets[i] - r_off; + const char* r_raw = reinterpret_cast(&rdata[r_off]); + JsonbPath path; + if (!path.seek(r_raw, r_size)) { + return Status::InvalidArgument( + "Json path error: {} for value: {}", + JsonbErrMsg::getErrMsg(JsonbErrType::E_INVALID_JSON_PATH), + std::string_view(reinterpret_cast(rdata.data()), + rdata.size())); + } + obj_val = doc->getValue()->findValue(path, nullptr); + } else { + obj_val = doc->getValue()->findValue(const_path, nullptr); + } + } else { + obj_val = doc->getValue(); + } + + if (!obj_val || !obj_val->isObject()) { + // if jsonb data is not object we should return null + res_null_map[i] = 1; + dst_arr.insert_default(); + continue; + } + ObjectVal* obj = (ObjectVal*)obj_val; + for (auto it = obj->begin(); it != obj->end(); ++it) { + dst_nested_column.insert_data(it->getKeyStr(), it->klen()); + } + dst_arr.get_offsets().push_back(dst_nested_column.size()); + } //for + return Status::OK(); + } +}; + class FunctionJsonbExtractPath : public IFunction { public: static constexpr auto name = "json_exists_path"; @@ -1468,6 +1639,9 @@ void register_function_jsonb(SimpleFunctionFactory& factory) { factory.register_function(); factory.register_alias(FunctionJsonbType::name, FunctionJsonbType::alias); + factory.register_function(); + factory.register_alias(FunctionJsonbKeys::name, FunctionJsonbKeys::alias); + factory.register_function(); factory.register_alias(FunctionJsonbExtractIsnull::name, FunctionJsonbExtractIsnull::alias); factory.register_function(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java index d4b8c99b536f19..20ec8ecd80f4b0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java @@ -222,6 +222,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonInsert; +import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonQuote; @@ -715,6 +716,7 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(JsonbType.class, "jsonb_type"), scalar(JsonLength.class, "json_length"), scalar(JsonContains.class, "json_contains"), + scalar(JsonKeys.class, "json_keys", "jsonb_keys"), scalar(L1Distance.class, "l1_distance"), scalar(L2Distance.class, "l2_distance"), scalar(LastDay.class, "last_day"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonKeys.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonKeys.java new file mode 100644 index 00000000000000..6c6cbed4c39c9c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/JsonKeys.java @@ -0,0 +1,84 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.scalar; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; +import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.ArrayType; +import org.apache.doris.nereids.types.JsonType; +import org.apache.doris.nereids.types.StringType; +import org.apache.doris.nereids.types.VarcharType; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * JsonKeys returns the keys from the top-level value of a JSON object as a JSON array, + * or, if a path argument is given, the top-level keys from the selected path. + * Returns NULL if any argument is NULL, the json_doc argument is not an object, + * or path, if given, does not locate an object. + * An error occurs if the json_doc argument is not a valid JSON document + * or the path argument is not a valid path expression. + */ +public class JsonKeys extends ScalarFunction + implements ExplicitlyCastableSignature, AlwaysNullable { + + public static final List SIGNATURES = ImmutableList.of( + FunctionSignature.ret(ArrayType.of(StringType.INSTANCE)) + .args(JsonType.INSTANCE, VarcharType.SYSTEM_DEFAULT), + FunctionSignature.ret(ArrayType.of(StringType.INSTANCE)).args(JsonType.INSTANCE)); + + /** + * constructor with one or two arguments. + */ + public JsonKeys(Expression arg0) { + super("json_keys", arg0); + } + + public JsonKeys(Expression arg0, Expression arg1) { + super("json_keys", arg0, arg1); + } + + /** + * withChildren. + */ + @Override + public JsonKeys withChildren(List children) { + Preconditions.checkArgument(children.size() == 1 || children.size() == 2); + if (children.size() == 1) { + return new JsonKeys(children.get(0)); + } else { + return new JsonKeys(children.get(0), children.get(1)); + } + } + + @Override + public List getSignatures() { + return SIGNATURES; + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitJsonKeys(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java index 33171511fd64e3..dbf083fc9145b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java @@ -225,6 +225,7 @@ import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonContains; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonExtract; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonInsert; +import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonKeys; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonLength; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonObject; import org.apache.doris.nereids.trees.expressions.functions.scalar.JsonQuote; @@ -1320,6 +1321,10 @@ default R visitJsonExtract(JsonExtract jsonExtract, C context) { return visitScalarFunction(jsonExtract, context); } + default R visitJsonKeys(JsonKeys jsonKeys, C context) { + return visitScalarFunction(jsonKeys, context); + } + default R visitJsonInsert(JsonInsert jsonInsert, C context) { return visitScalarFunction(jsonInsert, context); } diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py index 38c9f8ac886030..85e44c053b73ef 100644 --- a/gensrc/script/doris_builtins_functions.py +++ b/gensrc/script/doris_builtins_functions.py @@ -1777,6 +1777,11 @@ [['jsonb_type'], 'STRING', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'], [['jsonb_type'], 'STRING', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'], + [['jsonb_keys'], 'ARRAY_STRING', ['JSONB'], 'ALWAYS_NULLABLE'], + [['jsonb_keys'], 'ARRAY_STRING', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'], + [['json_keys'], 'ARRAY_STRING', ['JSONB'], 'ALWAYS_NULLABLE'], + [['json_keys'], 'ARRAY_STRING', ['JSONB', 'STRING'], 'ALWAYS_NULLABLE'], + [['jsonb_extract'], 'JSONB', ['JSONB', 'VARCHAR', '...'], 'ALWAYS_NULLABLE'], [['jsonb_extract'], 'JSONB', ['JSONB', 'STRING', '...'], 'ALWAYS_NULLABLE'], [['jsonb_extract_isnull'], 'BOOLEAN', ['JSONB', 'VARCHAR'], 'ALWAYS_NULLABLE'], diff --git a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out index 9e7eddb936eac3..5aba6171117167 100644 --- a/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out +++ b/regression-test/data/jsonb_p0/test_jsonb_load_and_function.out @@ -7726,3 +7726,108 @@ true 30 -9223372036854775808 false 31 18446744073709551615 false +-- !sql_json_keys -- +["k1", "k2"] + +-- !sql_json_keys -- +["a.b.c", "a"] + +-- !sql_json_keys -- +["k1.a1", "k2"] + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} [] +12 {"k1":"v31","k2":300} ["k1", "k2"] +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} ["k1", "k2", "a1"] +26 \N \N +27 {"k1":"v1","k2":200} ["k1", "k2"] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} ["a.b.c", "a"] +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +{"a.b.c":{"k1.a1":"v31","k2":300},"a":{}} $.a [] +{"a.b.c":{"k1.a1":"v31","k2":300}} $.a.b.c \N +{"a.b.c":{"k1.a1":"v31","k2":300},"a":{"k1.a1":1}} $.a ["k1.a1"] +["a","b"] $.a \N + diff --git a/regression-test/data/nereids_function_p0/scalar_function/J.out b/regression-test/data/nereids_function_p0/scalar_function/J.out index 25f57c51c168a3..0994891dc715a7 100644 --- a/regression-test/data/nereids_function_p0/scalar_function/J.out +++ b/regression-test/data/nereids_function_p0/scalar_function/J.out @@ -7464,3 +7464,108 @@ false -- !json_extract_string -- "v31" +-- !sql_json_keys -- +["k1", "k2"] + +-- !sql_json_keys -- +["a.b.c", "a"] + +-- !sql_json_keys -- +["k1.a1", "k2"] + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !sql_json_keys -- +\N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} [] +12 {"k1":"v31","k2":300} ["k1", "k2"] +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} ["k1", "k2", "a1"] +26 \N \N +27 {"k1":"v1","k2":200} ["k1", "k2"] +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} ["a.b.c", "a"] +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +1 \N \N +2 null \N +3 true \N +4 false \N +5 100 \N +6 10000 \N +7 1000000000 \N +8 1152921504606846976 \N +9 6.18 \N +10 "abcd" \N +11 {} \N +12 {"k1":"v31","k2":300} \N +13 [] \N +14 [123,456] \N +15 ["abc","def"] \N +16 [null,true,false,100,6.18,"abc"] \N +17 [{"k1":"v41","k2":400},1,"a",3.14] \N +18 {"k1":"v31","k2":300,"a1":[{"k1":"v41","k2":400},1,"a",3.14]} \N +26 \N \N +27 {"k1":"v1","k2":200} \N +28 {"a.b.c":{"k1.a1":"v31","k2":300},"a":"niu"} \N +29 12524337771678448270 \N +30 -9223372036854775808 \N +31 18446744073709551615 \N + +-- !select_json_keys -- +1 {"a.b.c":{"k1.a1":"v31","k2":300},"a":{}} $.a [] +2 {"a.b.c":{"k1.a1":"v31","k2":300}} $.a.b.c \N +3 {"a.b.c":{"k1.a1":"v31","k2":300},"a":{"k1.a1":1}} $.a ["k1.a1"] +4 ["a","b"] $.a \N + diff --git a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy index bc63071f66a115..2c9dcd6b2d124d 100644 --- a/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy +++ b/regression-test/suites/jsonb_p0/test_jsonb_load_and_function.groovy @@ -572,4 +572,45 @@ suite("test_jsonb_load_and_function", "p0") { qt_select_json_contains """SELECT id, j, json_contains(j, cast('{"k2":300}' as json)) FROM ${testTable} ORDER BY id""" qt_select_json_contains """SELECT id, j, json_contains(j, cast('{"k1":"v41","k2":400}' as json), '\$.a1') FROM ${testTable} ORDER BY id""" qt_select_json_contains """SELECT id, j, json_contains(j, cast('[123,456]' as json)) FROM ${testTable} ORDER BY id""" + + // json_keys + qt_sql_json_keys """SELECT json_keys('{"k1":"v31","k2":300}')""" + qt_sql_json_keys """SELECT json_keys('{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + qt_sql_json_keys """SELECT json_keys('{"a":{"k1.a1":"v31", "k2": 300},"b":"niu"}','\$.a')""" + qt_sql_json_keys """SELECT json_keys('abc','\$.k1')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]', '\$.k2')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]', '\$[1]')""" + + // error keys + test { + sql """ SELECT JSON_KEYS('{"a": {}, "a.b.c": {"c": 30}}', ''); """ + exception("Invalid Json Path for value") + } + + test { + sql """ SELECT JSON_KEYS('{"a": {}, "a.b.c": {"c": 30}}', 'a.b.c'); """ + exception("Invalid Json Path for value") + } + + // from table + qt_select_json_keys """SELECT id, j, json_keys(j) FROM ${testTable} ORDER BY id""" + qt_select_json_keys """SELECT id, j, json_keys(j, '\$.k2') FROM ${testTable} ORDER BY id""" + qt_select_json_keys """SELECT id, j, json_keys(j, '\$.a1') FROM ${testTable} ORDER BY id""" + + // make table with path + sql """ DROP TABLE IF EXISTS json_keys_table;""" + sql """ + CREATE TABLE IF NOT EXISTS json_keys_table ( + id INT, + j JSONB, + p STRING + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + sql """ insert into json_keys_table values (1, '{"a.b.c":{"k1.a1":"v31", "k2": 300}, "a": {}}', '\$.a'), (2, '{"a.b.c":{"k1.a1":"v31", "k2": 300}}', '\$.a.b.c'), (3, '{"a.b.c":{"k1.a1":"v31", "k2": 300}, "a": {"k1.a1": 1}}', '\$.a'), (4, '["a", "b"]', '\$.a'); """ + qt_select_json_keys """SELECT j, p, json_keys(j, p) FROM json_keys_table ORDER BY id""" } diff --git a/regression-test/suites/nereids_function_p0/scalar_function/J.groovy b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy index 939e70e5db6191..ecb2a9dc53105f 100644 --- a/regression-test/suites/nereids_function_p0/scalar_function/J.groovy +++ b/regression-test/suites/nereids_function_p0/scalar_function/J.groovy @@ -539,4 +539,46 @@ suite("nereids_scalar_fn_J") { qt_json_extract_string """select jsonb_extract('{"k1":"v31","k2":300}','\$.k1');""" + // json_keys + qt_sql_json_keys """SELECT json_keys('{"k1":"v31","k2":300}')""" + qt_sql_json_keys """SELECT json_keys('{"a.b.c":{"k1.a1":"v31", "k2": 300},"a":"niu"}')""" + qt_sql_json_keys """SELECT json_keys('{"a":{"k1.a1":"v31", "k2": 300},"b":"niu"}','\$.a')""" + qt_sql_json_keys """SELECT json_keys('abc','\$.k1')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]', '\$.k2')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]')""" + qt_sql_json_keys """SELECT json_keys('["a", "b", "c"]', '\$[1]')""" + + // error keys + test { + sql """ SELECT JSON_KEYS('{"a": {}, "a.b.c": {"c": 30}}', ''); """ + exception("Invalid Json Path for value") + } + + test { + sql """ SELECT JSON_KEYS('{"a": {}, "a.b.c": {"c": 30}}', 'a.b.c'); """ + exception("Invalid Json Path for value") + } + + // from table + qt_select_json_keys """SELECT id, j, json_keys(j) FROM ${testTable} ORDER BY id""" + qt_select_json_keys """SELECT id, j, json_keys(j, '\$.k2') FROM ${testTable} ORDER BY id""" + qt_select_json_keys """SELECT id, j, json_keys(j, '\$.a1') FROM ${testTable} ORDER BY id""" + + // make table with path + sql """ DROP TABLE IF EXISTS json_keys_table_nereid; """ + sql """ + CREATE TABLE IF NOT EXISTS json_keys_table_nereid ( + id INT, + j JSONB, + p STRING + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 10 + PROPERTIES("replication_num" = "1"); + """ + + sql """ insert into json_keys_table_nereid values (1, '{"a.b.c":{"k1.a1":"v31", "k2": 300}, "a": {}}', '\$.a'), (2, '{"a.b.c":{"k1.a1":"v31", "k2": 300}}', '\$.a.b.c'), (3, '{"a.b.c":{"k1.a1":"v31", "k2": 300}, "a": {"k1.a1": 1}}', '\$.a'), (4, '["a", "b"]', '\$.a'); """ + qt_select_json_keys """SELECT id, j, p, json_keys(j, p) FROM json_keys_table_nereid ORDER BY id""" + + }