Skip to content

Commit

Permalink
Support json_depth (#8435)
Browse files Browse the repository at this point in the history
close #8434
  • Loading branch information
SeaRise authored Nov 30, 2023
1 parent fbf4a35 commit 83c2256
Show file tree
Hide file tree
Showing 8 changed files with 221 additions and 2 deletions.
2 changes: 1 addition & 1 deletion dbms/src/Flash/Coprocessor/DAGUtils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -461,7 +461,7 @@ const std::unordered_map<tipb::ScalarFuncSig, String> scalar_func_map({
//{tipb::ScalarFuncSig::JsonQuoteSig, "cast"},
//{tipb::ScalarFuncSig::JsonSearchSig, "cast"},
//{tipb::ScalarFuncSig::JsonStorageSizeSig, "cast"},
//{tipb::ScalarFuncSig::JsonDepthSig, "cast"},
{tipb::ScalarFuncSig::JsonDepthSig, "json_depth"},
//{tipb::ScalarFuncSig::JsonKeysSig, "cast"},
{tipb::ScalarFuncSig::JsonLengthSig, "jsonLength"},
//{tipb::ScalarFuncSig::JsonKeys2ArgsSig, "cast"},
Expand Down
1 change: 1 addition & 0 deletions dbms/src/Functions/FunctionsJson.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -31,5 +31,6 @@ void registerFunctionsJson(FunctionFactory & factory)
factory.registerFunction<FunctionCastStringAsJson>();
factory.registerFunction<FunctionCastTimeAsJson>();
factory.registerFunction<FunctionCastDurationAsJson>();
factory.registerFunction<FunctionJsonDepth>();
}
} // namespace DB
66 changes: 66 additions & 0 deletions dbms/src/Functions/FunctionsJson.h
Original file line number Diff line number Diff line change
Expand Up @@ -1442,4 +1442,70 @@ class FunctionCastDurationAsJson : public IFunction
block.getByPosition(result).column = std::move(col_to);
}
};

class FunctionJsonDepth : public IFunction
{
public:
static constexpr auto name = "json_depth";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionJsonDepth>(); }

String getName() const override { return name; }

size_t getNumberOfArguments() const override { return 1; }

bool useDefaultImplementationForNulls() const override { return true; }
bool useDefaultImplementationForConstants() const override { return true; }

DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if unlikely (!arguments[0]->isString())
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}",
arguments[0]->getName(),
getName());
return std::make_shared<DataTypeUInt64>();
}

void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override
{
const auto & from = block.getByPosition(arguments[0]);
if (const auto * col_from = checkAndGetColumn<ColumnString>(from.column.get()))
{
const auto & data = col_from->getChars();
const auto & offsets = col_from->getOffsets();
const size_t size = offsets.size();

auto col_res = ColumnUInt64::create();
auto & vec_col_res = col_res->getData();
vec_col_res.resize(size);

ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
size_t data_length = offsets[i] - prev_offset - 1;
if (isNullJsonBinary(data_length))
{
vec_col_res[i] = 0;
}
else
{
JsonBinary json_binary(data[prev_offset], StringRef(&data[prev_offset + 1], data_length - 1));
vec_col_res[i] = json_binary.getDepth();
}
prev_offset = offsets[i];
}

block.getByPosition(result).column = std::move(col_res);
}
else
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}",
from.type->getName(),
getName());
}
}
};
} // namespace DB
2 changes: 1 addition & 1 deletion dbms/src/Functions/tests/gtest_cast_as_json.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ namespace DB::tests
* bool useDefaultImplementationForNulls() const override { return true; }
* bool useDefaultImplementationForConstants() const override { return true; }
* ```
* there is no need to test const, null_value, and only value.
* there is no need to test const, null_value, and only null.
*
* CastIntAsJson, CastStringAsJson and CastDurationAsJson can only test the case where input_tidb_tp/output_tidb_tp is nullptr
*/
Expand Down
84 changes: 84 additions & 0 deletions dbms/src/Functions/tests/gtest_json_depth.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
// 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 <Columns/ColumnNullable.h>
#include <TestUtils/FunctionTestUtils.h>
#include <TestUtils/TiFlashTestBasic.h>
#include <TiDB/Decode/JsonBinary.h>

#include <string>
#include <vector>

namespace DB::tests
{
/**
* Because FunctionJsonDepth have
* ```
* bool useDefaultImplementationForNulls() const override { return true; }
* bool useDefaultImplementationForConstants() const override { return true; }
* ```
* there is no need to test const, null_value, and only null.
*/
class TestJsonDepth : public DB::tests::FunctionTest
{
public:
ColumnWithTypeAndName executeFunctionWithCast(const ColumnWithTypeAndName & column)
{
// Use string as input column to improve readability.
assert(column.type->isString());
ColumnsWithTypeAndName origin_inputs{column};
auto json_column = executeFunction("cast_string_as_json", origin_inputs, nullptr, true);
return executeFunction("json_depth", json_column);
}

void executeAndAssert(const String & input, const UInt64 & expect)
{
ASSERT_COLUMN_EQ(createColumn<UInt64>({expect}), executeFunctionWithCast({createColumn<String>({input})}));
}
};

TEST_F(TestJsonDepth, TestAll)
try
{
// int
executeAndAssert("0", 1);
executeAndAssert("1", 1);
executeAndAssert("-1", 1);
// double
executeAndAssert("1.1111", 1);
executeAndAssert("-1.1111", 1);
// bool
executeAndAssert("true", 1);
executeAndAssert("false", 1);
// string
executeAndAssert("\"sdhfgjksdahfjksdhfjhsdjkfhjskdhfkjsdhfjksdhfkj\"", 1);
executeAndAssert("\"\"", 1);

// array
executeAndAssert("[]", 1);
executeAndAssert("[[]]", 2);
executeAndAssert("[[[[[[[[[[[[[[[[[[[[[]]]]]]]]]]]]]]]]]]]]]", 21);

// obj
executeAndAssert("{}", 1);
executeAndAssert("{\"a\":1}", 2);
executeAndAssert("{\"a\":{}}", 2);
executeAndAssert(R"({"a":{"a":{"a":{"a":{"a":{"a":{"a":{"a":{"a":{"a":{}}}}}}}}}}})", 11);

// complex
executeAndAssert(R"([{}, "a", 1, 1.232, {"a": ["a", [{"a": {"b":true}}]]}])", 7);
}
CATCH

} // namespace DB::tests
32 changes: 32 additions & 0 deletions dbms/src/TiDB/Decode/JsonBinary.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -835,6 +835,38 @@ bool JsonBinary::extract(
return found;
}

// same as https://github.com/pingcap/tidb/blob/4114da88a57be6ff7f985944a247811e8b3138c5/pkg/types/json_binary_functions.go#L1147-L1157
UInt64 JsonBinary::getDepth() const
{
switch (type)
{
case TYPE_CODE_OBJECT:
{
auto elem_count = getElementCount();
UInt64 max_depth = 0;
for (size_t i = 0; i < elem_count; ++i)
{
const auto & obj = getObjectValue(i);
max_depth = std::max(max_depth, obj.getDepth());
}
return max_depth + 1;
}
case TYPE_CODE_ARRAY:
{
auto elem_count = getElementCount();
UInt64 max_depth = 0;
for (size_t i = 0; i < elem_count; ++i)
{
const auto & obj = getArrayElement(i);
max_depth = std::max(max_depth, obj.getDepth());
}
return max_depth + 1;
}
default:
return 1;
}
}

std::optional<JsonBinary> JsonBinary::searchObjectKey(JsonPathObjectKey & key) const
{
auto element_count = getElementCount();
Expand Down
2 changes: 2 additions & 0 deletions dbms/src/TiDB/Decode/JsonBinary.h
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,8 @@ class JsonBinary
std::vector<JsonPathExprRefContainerPtr> & path_expr_container_vec,
JsonBinaryWriteBuffer & write_buffer);

UInt64 getDepth() const;

static String unquoteString(const StringRef & ref);
static void unquoteStringInBuffer(const StringRef & ref, JsonBinaryWriteBuffer & write_buffer);
static String unquoteJsonString(const StringRef & ref);
Expand Down
34 changes: 34 additions & 0 deletions tests/fullstack-test/expr/json_depth.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
# 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(col 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

mysql> set tidb_allow_mpp=1;set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select json_depth(col) from test.t;
+-----------------+
| json_depth(col) |
+-----------------+
| NULL |
| 2 |
| 3 |
| 3 |
| 1 |
+-----------------+

# Clean up.
mysql> drop table if exists test.t;

0 comments on commit 83c2256

Please sign in to comment.