diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 14e07091e8c..b97fada3444 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -448,7 +448,7 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::JsonRemoveSig, "cast"}, //{tipb::ScalarFuncSig::JsonMergeSig, "cast"}, //{tipb::ScalarFuncSig::JsonObjectSig, "cast"}, - //{tipb::ScalarFuncSig::JsonArraySig, "cast"}, + {tipb::ScalarFuncSig::JsonArraySig, "json_array"}, //{tipb::ScalarFuncSig::JsonValidJsonSig, "cast"}, //{tipb::ScalarFuncSig::JsonContainsSig, "cast"}, //{tipb::ScalarFuncSig::JsonArrayAppendSig, "cast"}, diff --git a/dbms/src/Functions/FunctionsJson.cpp b/dbms/src/Functions/FunctionsJson.cpp index ef9f725c1a6..4b3e01d836c 100644 --- a/dbms/src/Functions/FunctionsJson.cpp +++ b/dbms/src/Functions/FunctionsJson.cpp @@ -22,5 +22,7 @@ void registerFunctionsJson(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } // namespace DB diff --git a/dbms/src/Functions/FunctionsJson.h b/dbms/src/Functions/FunctionsJson.h index cee84310191..0efdf6b0a11 100644 --- a/dbms/src/Functions/FunctionsJson.h +++ b/dbms/src/Functions/FunctionsJson.h @@ -21,10 +21,13 @@ #include #include #include +#include #include #include #include +#include + namespace DB { /** Json related functions: @@ -34,6 +37,8 @@ namespace DB * Throw exception if any path_string failed to parse. * json_unquote(json_string) * cast_json_as_string(json_object) + * json_length(json_object) + * json_array(json_object...) * */ @@ -47,6 +52,8 @@ inline bool isNullJsonBinary(size_t size) return size == 0; } +using namespace GatherUtils; + class FunctionsJsonExtract : public IFunction { public: @@ -324,6 +331,7 @@ class FunctionsJsonUnquote : public IFunction } }; + class FunctionsCastJsonAsString : public IFunction { public: @@ -392,4 +400,143 @@ class FunctionsCastJsonAsString : public IFunction } }; -} // namespace DB \ No newline at end of file + +class FunctionJsonLength : public IFunction +{ +public: + static constexpr auto name = "jsonLength"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!arguments[0]->isString()) + throw Exception( + fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const auto * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnUInt64::create(); + typename ColumnUInt64::Container & vec_col_res = col_res->getData(); + { + const auto & data = col->getChars(); + const auto & offsets = col->getOffsets(); + const size_t size = offsets.size(); + vec_col_res.resize(size); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + std::string_view sv( + reinterpret_cast(&data[prev_offset]), + offsets[i] - prev_offset - 1); + vec_col_res[i] = JsonBinary::getJsonLength(sv); + prev_offset = offsets[i]; + } + } + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + fmt::format("Illegal column {} of argument of function {}", column->getName(), getName()), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +class FunctionsJsonArray : public IFunction +{ +public: + static constexpr auto name = "json_array"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isVariadic() const override { return true; } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + for (const auto arg_idx : ext::range(0, arguments.size())) + { + if (!arguments[arg_idx]->onlyNull()) + { + const auto * arg = removeNullable(arguments[arg_idx]).get(); + if (!arg->isStringOrFixedString()) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arg->getName(), + arg_idx + 1, + getName()); + } + } + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override + { + auto nested_block = createBlockWithNestedColumns(block, arguments, result); + StringSources sources; + for (auto column_number : arguments) + { + sources.push_back( + block.getByPosition(column_number).type->onlyNull() + ? nullptr + : createDynamicStringSource(*nested_block.getByPosition(column_number).column)); + } + + auto rows = block.rows(); + auto col_to = ColumnString::create(); + auto & data_to = col_to->getChars(); + WriteBufferFromVector write_buffer(data_to); + auto & offsets_to = col_to->getOffsets(); + offsets_to.resize(rows); + + std::vector jsons; + jsons.reserve(sources.size()); + for (size_t i = 0; i < rows; ++i) + { + for (size_t col = 0; col < sources.size(); ++col) + { + if (sources[col] && !block.getByPosition(arguments[col]).column->isNullAt(i)) + { + const auto & data_from = sources[col]->getWhole(); + jsons.emplace_back(data_from.data[0], StringRef(&data_from.data[1], data_from.size - 1)); + } + else + { + jsons.emplace_back(JsonBinary::TYPE_CODE_LITERAL, StringRef(&JsonBinary::LITERAL_NIL, 1)); + } + } + JsonBinary::buildBinaryJsonArrayInBuffer(jsons, write_buffer); + jsons.clear(); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + for (const auto & source : sources) + { + if (source) + source->next(); + } + } + data_to.resize(write_buffer.count()); + + block.getByPosition(result).column = std::move(col_to); + } +}; +} // namespace DB diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index 3cccf5dcb08..70c25e7d709 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -1102,62 +1102,6 @@ class FunctionReverse : public IFunction } }; -extern UInt64 GetJsonLength(const std::string_view & sv); - -class FunctionJsonLength : public IFunction -{ -public: - static constexpr auto name = "jsonLength"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!arguments[0]->isString()) - throw Exception( - fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const auto * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnUInt64::create(); - typename ColumnUInt64::Container & vec_col_res = col_res->getData(); - { - const auto & data = col->getChars(); - const auto & offsets = col->getOffsets(); - const size_t size = offsets.size(); - vec_col_res.resize(size); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - std::string_view sv( - reinterpret_cast(&data[prev_offset]), - offsets[i] - prev_offset - 1); - vec_col_res[i] = GetJsonLength(sv); - prev_offset = offsets[i]; - } - } - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - fmt::format("Illegal column {} of argument of function {}", column->getName(), getName()), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - template class ConcatImpl : public IFunction { @@ -6563,7 +6507,6 @@ void registerFunctionsString(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/tests/gtest_json_array.cpp b/dbms/src/Functions/tests/gtest_json_array.cpp new file mode 100644 index 00000000000..caa4a51ac88 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_json_array.cpp @@ -0,0 +1,154 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed 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. + +#include +#include +#include +#include + +#include +#include + +namespace DB::tests +{ +class TestJsonArray : public DB::tests::FunctionTest +{ +public: + ColumnWithTypeAndName executeFunctionWithCast( + const ColumnNumbers & argument_column_numbers, + const ColumnsWithTypeAndName & columns) + { + auto json_column = executeFunction("json_array", argument_column_numbers, columns); + static auto json_array_return_type = std::make_shared(); + assert(json_array_return_type->equals(*json_column.type)); + // The `json_binary` should be cast as a string to improve readability. + return executeFunction("cast_json_as_string", {json_column}); + } +}; + +TEST_F(TestJsonArray, TestAll) +try +{ + auto const not_null_json_type = std::make_shared(); + auto const nullable_json_type = makeNullable(std::make_shared()); + + /// prepare has value json column/const + // [] + // clang-format off + const UInt8 empty_array[] = { + JsonBinary::TYPE_CODE_ARRAY, // array_type + 0x0, 0x0, 0x0, 0x0, // element_count + 0x8, 0x0, 0x0, 0x0}; // total_size + const size_t rows_count = 2; + // clang-format on + ColumnWithTypeAndName not_null_column; + { + auto empty_array_json = ColumnString::create(); + empty_array_json->insertData(reinterpret_cast(empty_array), sizeof(empty_array) / sizeof(UInt8)); + empty_array_json->insertData(reinterpret_cast(empty_array), sizeof(empty_array) / sizeof(UInt8)); + not_null_column = ColumnWithTypeAndName(std::move(empty_array_json), not_null_json_type); + } + ColumnWithTypeAndName not_null_const; + { + auto empty_array_json = ColumnString::create(); + empty_array_json->insertData(reinterpret_cast(empty_array), sizeof(empty_array) / sizeof(UInt8)); + auto const_col = ColumnConst::create(std::move(empty_array_json), rows_count); + not_null_const = ColumnWithTypeAndName(std::move(const_col), not_null_json_type); + } + ColumnWithTypeAndName nullable_but_not_null_column; + { + auto empty_array_json = ColumnString::create(); + empty_array_json->insertData(reinterpret_cast(empty_array), sizeof(empty_array) / sizeof(UInt8)); + empty_array_json->insertData(reinterpret_cast(empty_array), sizeof(empty_array) / sizeof(UInt8)); + ColumnUInt8::MutablePtr col_null_map = ColumnUInt8::create(rows_count, 0); + auto json_col = ColumnNullable::create(std::move(empty_array_json), std::move(col_null_map)); + nullable_but_not_null_column = ColumnWithTypeAndName(std::move(json_col), nullable_json_type); + } + + /// prepare null value json column/const + ColumnWithTypeAndName null_column; + { + auto str_col = ColumnString::create(); + str_col->insertData("", 0); + str_col->insertData("", 0); + ColumnUInt8::MutablePtr col_null_map = ColumnUInt8::create(rows_count, 1); + auto json_col = ColumnNullable::create(std::move(str_col), std::move(col_null_map)); + null_column = ColumnWithTypeAndName(std::move(json_col), nullable_json_type); + } + ColumnWithTypeAndName null_const = createConstColumn>(rows_count, {}); + + /// prepare only null column + ColumnWithTypeAndName only_null_const = createOnlyNullColumnConst(rows_count); + + /// prepare input columns + ColumnsWithTypeAndName inputs( + {not_null_column, not_null_const, nullable_but_not_null_column, null_column, null_const, only_null_const}); + + auto gen_column_expect = [](const String & value) { + // double for rows_count 2. + return createColumn>({value, value}); + }; + + // json_array() + { + auto res = executeFunctionWithCast({}, inputs); + auto expect = gen_column_expect("[]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(all columns) + { + auto res = executeFunctionWithCast({0, 1, 2, 3, 4, 5}, inputs); + auto expect = gen_column_expect("[[], [], [], null, null, null]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(only_null, only_null) + { + auto res = executeFunctionWithCast({5, 5}, inputs); + auto expect = createConstColumn>(rows_count, "[null, null]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(null_const, not_null_const, only_null) + { + auto res = executeFunctionWithCast({4, 1, 5}, inputs); + auto expect = createConstColumn>(rows_count, "[null, [], null]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(not_null_const, not_null_const) + { + auto res = executeFunctionWithCast({1, 1}, inputs); + auto expect = createConstColumn>(rows_count, "[[], []]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(not_null_column, null_column) + { + auto res = executeFunctionWithCast({0, 3}, inputs); + auto expect = gen_column_expect("[[], null]"); + ASSERT_COLUMN_EQ(expect, res); + } + + // json_array(not_null_column, not_null_const) + { + auto res = executeFunctionWithCast({0, 1}, inputs); + auto expect = gen_column_expect("[[], []]"); + ASSERT_COLUMN_EQ(expect, res); + } +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/TiDB/Decode/JsonBinary.cpp b/dbms/src/TiDB/Decode/JsonBinary.cpp index d5bf9d97395..68562c5d268 100644 --- a/dbms/src/TiDB/Decode/JsonBinary.cpp +++ b/dbms/src/TiDB/Decode/JsonBinary.cpp @@ -823,7 +823,7 @@ void JsonBinary::buildBinaryJsonArrayInBuffer( buildBinaryJsonElementsInBuffer(json_binary_vec, write_buffer); } -UInt64 GetJsonLength(const std::string_view & raw_value) +UInt64 JsonBinary::getJsonLength(const std::string_view & raw_value) { if (raw_value.empty()) { diff --git a/dbms/src/TiDB/Decode/JsonBinary.h b/dbms/src/TiDB/Decode/JsonBinary.h index 53b35e89663..cb7b1a663e0 100644 --- a/dbms/src/TiDB/Decode/JsonBinary.h +++ b/dbms/src/TiDB/Decode/JsonBinary.h @@ -122,6 +122,11 @@ class JsonBinary /// Opaque represents a raw database binary type struct Opaque { + Opaque(UInt8 type_, const StringRef & data_) + : type(type_) + , data(data_) + {} + // TypeCode is the same with TiDB database type code UInt8 type; // Buf is the underlying bytes of the data @@ -152,6 +157,12 @@ class JsonBinary static void SkipJson(size_t & cursor, const String & raw_value); static String DecodeJsonAsBinary(size_t & cursor, const String & raw_value); + static void buildBinaryJsonArrayInBuffer( + const std::vector & json_binary_vec, + JsonBinaryWriteBuffer & write_buffer); + + static UInt64 getJsonLength(const std::string_view & raw_value); + private: Int64 getInt64() const; UInt64 getUInt64() const; @@ -186,9 +197,6 @@ class JsonBinary static void buildBinaryJsonElementsInBuffer( const std::vector & json_binary_vec, JsonBinaryWriteBuffer & write_buffer); - static void buildBinaryJsonArrayInBuffer( - const std::vector & json_binary_vec, - JsonBinaryWriteBuffer & write_buffer); static void marshalFloat64To(JsonBinaryWriteBuffer & write_buffer, double f); static void marshalLiteralTo(JsonBinaryWriteBuffer & write_buffer, UInt8 lit_type); static void marshalStringTo(JsonBinaryWriteBuffer & write_buffer, const StringRef & ref); diff --git a/tests/fullstack-test/expr/json_array.test b/tests/fullstack-test/expr/json_array.test new file mode 100644 index 00000000000..753014bc738 --- /dev/null +++ b/tests/fullstack-test/expr/json_array.test @@ -0,0 +1,85 @@ +# Copyright 2023 PingCAP, Inc. +# +# Licensed 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. + +mysql> drop table if exists test.t +mysql> create table test.t(a json, b json, c json, d json, e json) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values(null, '{\"a\" : \"b\", \"aa\" : \"bb\"}', '[1,2,[3,4]]', '[1,2,{\"a\":\"b\"}]', '\"hello world\"') #NO_UNESCAPE + +func> wait_table test t + +# json_array(null_value in column) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(a) from test.t; ++---------------+ +| json_array(a) | ++---------------+ +| [null] | ++---------------+ + +# json_array(not_null_value in column) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(b) from test.t; ++--------------------------+ +| json_array(b) | ++--------------------------+ +| [{"a": "b", "aa": "bb"}] | ++--------------------------+ + +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(b, c, d, e) from test.t; ++-----------------------------------------------------------------------------+ +| json_array(b, c, d, e) | ++-----------------------------------------------------------------------------+ +| [{"a": "b", "aa": "bb"}, [1, 2, [3, 4]], [1, 2, {"a": "b"}], "hello world"] | ++-----------------------------------------------------------------------------+ + +# json_array(null_value in column + not_null_value in column) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(a, b, c, d, e) from test.t; ++-----------------------------------------------------------------------------------+ +| json_array(a, b, c, d, e) | ++-----------------------------------------------------------------------------------+ +| [null, {"a": "b", "aa": "bb"}, [1, 2, [3, 4]], [1, 2, {"a": "b"}], "hello world"] | ++-----------------------------------------------------------------------------------+ + +# json_array(null_value in column + only_null_const) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(a, null) from test.t; ++---------------------+ +| json_array(a, null) | ++---------------------+ +| [null, null] | ++---------------------+ + +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(null, a) from test.t; ++---------------------+ +| json_array(null, a) | ++---------------------+ +| [null, null] | ++---------------------+ + +# json_array(not_null_value in column + only_null_const) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(null, b, c, null, null, d, e, null) from test.t; ++-----------------------------------------------------------------------------------------------------+ +| json_array(null, b, c, null, null, d, e, null) | ++-----------------------------------------------------------------------------------------------------+ +| [null, {"a": "b", "aa": "bb"}, [1, 2, [3, 4]], null, null, [1, 2, {"a": "b"}], "hello world", null] | ++-----------------------------------------------------------------------------------------------------+ + +# json_array(null_value in column + not_null_value in column + only_null_const) +mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_array(null, a, null, b, c, null, null, d, e, null) from test.t; ++-----------------------------------------------------------------------------------------------------------------+ +| json_array(null, a, null, b, c, null, null, d, e, null) | ++-----------------------------------------------------------------------------------------------------------------+ +| [null, null, null, {"a": "b", "aa": "bb"}, [1, 2, [3, 4]], null, null, [1, 2, {"a": "b"}], "hello world", null] | ++-----------------------------------------------------------------------------------------------------------------+ + +# Clean up. +mysql> drop table if exists test.t;